[
https://issues.apache.org/jira/browse/HBASE-23881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17047926#comment-17047926
]
Josh Elser commented on HBASE-23881:
------------------------------------
The problem seems to be that the SASL handshake does not fail as we expect. The
server will write back a response indicating that the {{SaslStatus.ERROR}}
occurred (rather than success). This is done as "raw" RPC, rather than our
protobuf RPC layer.
However, the caller doesn't error out at the SASL portion of the RPC, and
ultimately thinks the server's response is a protobuf response:
{noformat}
2020-02-28 15:16:44,925 TRACE [Default-IPC-NioEventLoopGroup-4-6]
ipc.AbstractRpcClient(371): Call: GetTableDescriptors, callTime: 41ms,
error=null, response=,
call=Call[id=0,methodName=GetTableDescriptors,param=TODO: class
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos$GetTableDescriptorsRequest]
{noformat}
This is why the client doesn't think anything went wrong. Something is
definitely wrong with the Netty+SASL code in master or there is something
non-intuitive server-side which the server-RPC layer requires (e.g. some
special method to be called?). We can easily see that the exception flows up
through the Netty server
{noformat}
2020-02-28 15:16:44,925 TRACE [RS-EventLoopGroup-1-6]
ipc.NettyRpcServerRequestDecoder(76): Connection /192.168.2.28:63311; caught
unexpected downstream exception.
org.apache.hadoop.security.token.SecretManager$InvalidToken: Authentication
failed for user1
at
org.apache.hadoop.hbase.security.provider.example.ShadeSaslServerAuthenticationProvider$ShadeSaslServerCallbackHandler.handle(ShadeSaslServerAuthenticationProvider.java:172)
at
org.apache.hadoop.hbase.security.provider.example.SaslPlainServer.evaluateResponse(SaslPlainServer.java:108)
at
org.apache.hadoop.hbase.security.HBaseSaslRpcServer.evaluateResponse(HBaseSaslRpcServer.java:65)
at
org.apache.hadoop.hbase.ipc.ServerRpcConnection.saslReadAndProcess(ServerRpcConnection.java:364)
at
org.apache.hadoop.hbase.ipc.NettyServerRpcConnection.process(NettyServerRpcConnection.java:87)
at
org.apache.hadoop.hbase.ipc.NettyServerRpcConnection.process(NettyServerRpcConnection.java:73)
at
org.apache.hadoop.hbase.ipc.NettyServerRpcConnection.process(NettyServerRpcConnection.java:68)
at
org.apache.hadoop.hbase.ipc.NettyRpcServerRequestDecoder.channelRead(NettyRpcServerRequestDecoder.java:62)
at
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:377)
at
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:363)
at
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:355)
at
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:316)
at
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:290)
at
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:377)
at
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:363)
at
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:355)
at
org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
at
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:377)
at
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:363)
at
org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
at
org.apache.hbase.thirdparty.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:163)
at
org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:714)
at
org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:650)
at
org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:576)
at
org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:493)
at
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
at
org.apache.hbase.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
at
org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.lang.Thread.run(Thread.java:748) {noformat}
> TestShadeSaslAuthenticationProvider failures
> --------------------------------------------
>
> Key: HBASE-23881
> URL: https://issues.apache.org/jira/browse/HBASE-23881
> Project: HBase
> Issue Type: Bug
> Components: test
> Affects Versions: 3.0.0, 2.3.0
> Reporter: Bharath Vissapragada
> Assignee: Josh Elser
> Priority: Major
>
> TestShadeSaslAuthenticationProvider now fails deterministically with the
> following exception..
> {noformat}
> java.lang.Exception: Unexpected exception,
> expected<org.apache.hadoop.hbase.DoNotRetryIOException> but
> was<java.io.IOException>
> at
> org.apache.hadoop.hbase.security.provider.example.TestShadeSaslAuthenticationProvider.testNegativeAuthentication(TestShadeSaslAuthenticationProvider.java:233)
> {noformat}
> The test now fails a different place than before merging HBASE-18095 because
> the RPCs are also a part of connection setup. We might need to rewrite the
> test..
--
This message was sent by Atlassian Jira
(v8.3.4#803005)