[ 
https://issues.apache.org/jira/browse/HBASE-28752?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

SunQiang updated HBASE-28752:
-----------------------------
    Description: 
Our HBase system is used for OLAP , The client has strict requirements for 
latency and stability, and the client configuration is as follows:
{code:java}
hbase.rpc.timeout: 100
hbase.client.operation.timeout: 500
hbase.client.retries.number: 3
hbase.client.pause: 120 {code}
When I logged off the Datanode, I received this exception:
{code:java}
2024-06-03 17:19:16,535 WARN  
[RpcServer.default.RWQ.Fifo.read.handler=216,queue=4,port=16020] 
hdfs.BlockReaderFactory: I/O error constructing remote block reader.
org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while 
waiting for channel to be ready for connect. ch : 
java.nio.channels.SocketChannel[connection-pending remote=/10.111.242.219:50010]
    at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:534)
    at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:3436)
    at 
org.apache.hadoop.hdfs.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:777)
    at 
org.apache.hadoop.hdfs.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:694)
    at 
org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:355)
    at 
org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1173)
    at org.apache.hadoop.hdfs.DFSInputStream.access$200(DFSInputStream.java:92)
    at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1118)
    at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1110)
    at java.util.concurrent.FutureTask.run(FutureTask.java:266)
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
    at java.util.concurrent.FutureTask.run(FutureTask.java:266)
    at 
java.util.concurrent.ThreadPoolExecutor$CallerRunsPolicy.rejectedExecution(ThreadPoolExecutor.java:2022)
    at org.apache.hadoop.hdfs.DFSClient$2.rejectedExecution(DFSClient.java:3481)
    at 
java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:823)
    at 
java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1369)
    at 
java.util.concurrent.ExecutorCompletionService.submit(ExecutorCompletionService.java:181)
    at 
org.apache.hadoop.hdfs.DFSInputStream.hedgedFetchBlockByteRange(DFSInputStream.java:1297)
 {code}
This will cause the HBase service to become unstable because HBase has accessed 
an offline datanode node, resulting in a long time required to create a socket 
connection to the offline datanode. Through stack logs, I found that it is 
controlled through the configuration of hdfs.client.socket timeout.

–

In hbase-site.xml,I found that adjusting the 
{color:#ff0000}hdfs.client.socket-time{color} configuration is effective,so I 
turned down the hdfs.client.socket time configuration from 60s to 5s. but I 
found that if I continued to turn down the hdfs.client.socket time 
configuration to {color:#ff0000}200ms{color}, the following exception occurred:
{code:java}
2024-06-18 15:51:24,212 WARN  [AsyncFSWAL-0] wal.AsyncFSWAL: sync failed
java.io.IOException: Timeout(200ms) waiting for response
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.lambda$userEventTriggered$4(FanOutOneBlockAsyncDFSOutput.java:302)
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.failed(FanOutOneBlockAsyncDFSOutput.java:235)
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.access$300(FanOutOneBlockAsyncDFSOutput.java:98)
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.userEventTriggered(FanOutOneBlockAsyncDFSOutput.java:301)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
        at 
org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.userEventTriggered(ChannelInboundHandlerAdapter.java:108)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
        at 
org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.userEventTriggered(ChannelInboundHandlerAdapter.java:108)
        at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.userEventTriggered(ByteToMessageDecoder.java:366)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
        at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelIdle(IdleStateHandler.java:371)
        at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler$ReaderIdleTimeoutTask.run(IdleStateHandler.java:494)
        at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler$AbstractIdleTask.run(IdleStateHandler.java:466)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.PromiseTask$RunnableAdapter.call(PromiseTask.java:38)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.ScheduledFutureTask.run(ScheduledFutureTask.java:127)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
        at 
org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:333)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:905)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:745){code}
 

The configuration of 'hdfs. client. socket time' is reused in the 
FanOutOneBlockAsyncDFSOutput.class of hbase.

–

In the 'FanOutOneBlockAsyncDFSOutput' construction method:
{code:java}
FanOutOneBlockAsyncDFSOutput(Configuration conf, FSUtils fsUtils, 
DistributedFileSystem dfs,
    DFSClient client, ClientProtocol namenode, String clientName, String src, 
long fileId,
    LocatedBlock locatedBlock, Encryptor encryptor, List<Channel> datanodeList,
    DataChecksum summer, ByteBufAllocator alloc) {
  this.conf = conf;
  this.fsUtils = fsUtils;
  this.dfs = dfs;
  this.client = client;
  this.namenode = namenode;
  this.fileId = fileId;
  this.clientName = clientName;
  this.src = src;
  this.block = locatedBlock.getBlock();
  this.locations = locatedBlock.getLocations();
  this.encryptor = encryptor;
  this.datanodeList = datanodeList;
  this.summer = summer;
  this.maxDataLen = MAX_DATA_LEN - (MAX_DATA_LEN % 
summer.getBytesPerChecksum());
  this.alloc = alloc;
  this.buf = alloc.directBuffer(sendBufSizePRedictor.initialSize());
  this.state = State.STREAMING;
*  setupReceiver(conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, READ_TIMEOUT));
} {code}
----
My implementation process:
1. add a new configuration in hbase-site.xml
{code:java}
+ <property>
+ <name>hbase.wal.asyncfsoutput.timeout</name>
+ <value>60000</value>
+ </property> {code}
2.modify code
{code:java}
151 + private static final String FANOUT_TIMEOUTKEY = 
"hbase.wal.asyncfsoutput.timeout";
339 - setupReceiver(conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, READ_TIMEOUT));
339 + setupReceiver(conf.getInt(FANOUT_TIMEOUTKEY, READ_TIMEOUT));
 {code}
3.repackge
 
—
I would like to consult the community on whether it is possible to design a 
separate configuration item in FanOutOneBlockAsynchronousDFSOutput.jva to 
isolate the issue of writing Wal timeout failure due to the small size of 
dfs.comient.socket time?

  was:
Our HBase system is used for OLAP , The client has strict requirements for 
latency and stability, and the client configuration is as follows:
{code:java}
hbase.rpc.timeout: 100
hbase.client.operation.timeout: 500
hbase.client.retries.number: 3
hbase.client.pause: 120 {code}
When I logged off the Datanode, I received this exception:
{code:java}
2024-06-03 17:19:16,535 WARN  
[RpcServer.default.RWQ.Fifo.read.handler=216,queue=4,port=16020] 
hdfs.BlockReaderFactory: I/O error constructing remote block reader.
org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while 
waiting for channel to be ready for connect. ch : 
java.nio.channels.SocketChannel[connection-pending remote=/10.111.242.219:50010]
    at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:534)
    at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:3436)
    at 
org.apache.hadoop.hdfs.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:777)
    at 
org.apache.hadoop.hdfs.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:694)
    at 
org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:355)
    at 
org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1173)
    at org.apache.hadoop.hdfs.DFSInputStream.access$200(DFSInputStream.java:92)
    at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1118)
    at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1110)
    at java.util.concurrent.FutureTask.run(FutureTask.java:266)
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
    at java.util.concurrent.FutureTask.run(FutureTask.java:266)
    at 
java.util.concurrent.ThreadPoolExecutor$CallerRunsPolicy.rejectedExecution(ThreadPoolExecutor.java:2022)
    at org.apache.hadoop.hdfs.DFSClient$2.rejectedExecution(DFSClient.java:3481)
    at 
java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:823)
    at 
java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1369)
    at 
java.util.concurrent.ExecutorCompletionService.submit(ExecutorCompletionService.java:181)
    at 
org.apache.hadoop.hdfs.DFSInputStream.hedgedFetchBlockByteRange(DFSInputStream.java:1297)
 {code}
This will cause the HBase service to become unstable because HBase has accessed 
an offline datanode node, resulting in a long time required to create a socket 
connection to the offline datanode. Through stack logs, I found that it is 
controlled through the configuration of hdfs.client.socket timeout.

–

In hbase-site.xml,I found that adjusting the 
{color:#ff0000}hdfs.client.socket-time{color} configuration is effective,so I 
turned down the hdfs.client.socket time configuration from 60s to 5s. but I 
found that if I continued to turn down the hdfs.client.socket time 
configuration to {color:#ff0000}200ms{color}, the following exception occurred:
{code:java}
2024-06-18 15:51:24,212 WARN  [AsyncFSWAL-0] wal.AsyncFSWAL: sync failed
java.io.IOException: Timeout(200ms) waiting for response
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.lambda$userEventTriggered$4(FanOutOneBlockAsyncDFSOutput.java:302)
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.failed(FanOutOneBlockAsyncDFSOutput.java:235)
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.access$300(FanOutOneBlockAsyncDFSOutput.java:98)
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.userEventTriggered(FanOutOneBlockAsyncDFSOutput.java:301)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
        at 
org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.userEventTriggered(ChannelInboundHandlerAdapter.java:108)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
        at 
org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.userEventTriggered(ChannelInboundHandlerAdapter.java:108)
        at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.userEventTriggered(ByteToMessageDecoder.java:366)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
        at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelIdle(IdleStateHandler.java:371)
        at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler$ReaderIdleTimeoutTask.run(IdleStateHandler.java:494)
        at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler$AbstractIdleTask.run(IdleStateHandler.java:466)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.PromiseTask$RunnableAdapter.call(PromiseTask.java:38)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.ScheduledFutureTask.run(ScheduledFutureTask.java:127)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
        at 
org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:333)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:905)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:745){code}
 

The configuration of 'hdfs. client. socket time' is reused in the 
FanOutOneBlockAsyncDFSOutput.class of hbase.

–

In the 'FanOutOneBlockAsyncDFSOutput' construction method:
{code:java}
FanOutOneBlockAsyncDFSOutput(Configuration conf, FSUtils fsUtils, 
DistributedFileSystem dfs,
    DFSClient client, ClientProtocol namenode, String clientName, String src, 
long fileId,
    LocatedBlock locatedBlock, Encryptor encryptor, List<Channel> datanodeList,
    DataChecksum summer, ByteBufAllocator alloc) {
  this.conf = conf;
  this.fsUtils = fsUtils;
  this.dfs = dfs;
  this.client = client;
  this.namenode = namenode;
  this.fileId = fileId;
  this.clientName = clientName;
  this.src = src;
  this.block = locatedBlock.getBlock();
  this.locations = locatedBlock.getLocations();
  this.encryptor = encryptor;
  this.datanodeList = datanodeList;
  this.summer = summer;
  this.maxDataLen = MAX_DATA_LEN - (MAX_DATA_LEN % 
summer.getBytesPerChecksum());
  this.alloc = alloc;
  this.buf = alloc.directBuffer(sendBufSizePRedictor.initialSize());
  this.state = State.STREAMING;
  setupReceiver(conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, READ_TIMEOUT));
} {code}
----
My implementation process:
1. add a new configuration in hbase-site.xml
{code:java}
+ <property>
+ <name>hbase.wal.asyncfsoutput.timeout</name>
+ <value>60000</value>
+ </property> {code}
2.modify code
{code:java}
151 + private static final String FANOUT_TIMEOUTKEY = 
"hbase.wal.asyncfsoutput.timeout";
339 - setupReceiver(conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, READ_TIMEOUT));
339 + setupReceiver(conf.getInt(FANOUT_TIMEOUTKEY, READ_TIMEOUT));
 {code}
3.repackge
 
—
I would like to consult the community on whether it is possible to design a 
separate configuration item in FanOutOneBlockAsynchronousDFSOutput.jva to 
isolate the issue of writing Wal timeout failure due to the small size of 
dfs.comient.socket time?


> wal.AsyncFSWAL: sync failed
> ---------------------------
>
>                 Key: HBASE-28752
>                 URL: https://issues.apache.org/jira/browse/HBASE-28752
>             Project: HBase
>          Issue Type: Improvement
>          Components: asyncclient, wal
>    Affects Versions: 2.1.10, 2.2.5
>            Reporter: SunQiang
>            Priority: Major
>
> Our HBase system is used for OLAP , The client has strict requirements for 
> latency and stability, and the client configuration is as follows:
> {code:java}
> hbase.rpc.timeout: 100
> hbase.client.operation.timeout: 500
> hbase.client.retries.number: 3
> hbase.client.pause: 120 {code}
> When I logged off the Datanode, I received this exception:
> {code:java}
> 2024-06-03 17:19:16,535 WARN  
> [RpcServer.default.RWQ.Fifo.read.handler=216,queue=4,port=16020] 
> hdfs.BlockReaderFactory: I/O error constructing remote block reader.
> org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while 
> waiting for channel to be ready for connect. ch : 
> java.nio.channels.SocketChannel[connection-pending 
> remote=/10.111.242.219:50010]
>     at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:534)
>     at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:3436)
>     at 
> org.apache.hadoop.hdfs.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:777)
>     at 
> org.apache.hadoop.hdfs.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:694)
>     at 
> org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:355)
>     at 
> org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1173)
>     at 
> org.apache.hadoop.hdfs.DFSInputStream.access$200(DFSInputStream.java:92)
>     at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1118)
>     at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1110)
>     at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>     at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>     at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>     at 
> java.util.concurrent.ThreadPoolExecutor$CallerRunsPolicy.rejectedExecution(ThreadPoolExecutor.java:2022)
>     at 
> org.apache.hadoop.hdfs.DFSClient$2.rejectedExecution(DFSClient.java:3481)
>     at 
> java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:823)
>     at 
> java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1369)
>     at 
> java.util.concurrent.ExecutorCompletionService.submit(ExecutorCompletionService.java:181)
>     at 
> org.apache.hadoop.hdfs.DFSInputStream.hedgedFetchBlockByteRange(DFSInputStream.java:1297)
>  {code}
> This will cause the HBase service to become unstable because HBase has 
> accessed an offline datanode node, resulting in a long time required to 
> create a socket connection to the offline datanode. Through stack logs, I 
> found that it is controlled through the configuration of hdfs.client.socket 
> timeout.
> –
> In hbase-site.xml,I found that adjusting the 
> {color:#ff0000}hdfs.client.socket-time{color} configuration is effective,so I 
> turned down the hdfs.client.socket time configuration from 60s to 5s. but I 
> found that if I continued to turn down the hdfs.client.socket time 
> configuration to {color:#ff0000}200ms{color}, the following exception 
> occurred:
> {code:java}
> 2024-06-18 15:51:24,212 WARN  [AsyncFSWAL-0] wal.AsyncFSWAL: sync failed
> java.io.IOException: Timeout(200ms) waiting for response
>         at 
> org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.lambda$userEventTriggered$4(FanOutOneBlockAsyncDFSOutput.java:302)
>         at 
> org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.failed(FanOutOneBlockAsyncDFSOutput.java:235)
>         at 
> org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.access$300(FanOutOneBlockAsyncDFSOutput.java:98)
>         at 
> org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.userEventTriggered(FanOutOneBlockAsyncDFSOutput.java:301)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.userEventTriggered(ChannelInboundHandlerAdapter.java:108)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.userEventTriggered(ChannelInboundHandlerAdapter.java:108)
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.userEventTriggered(ByteToMessageDecoder.java:366)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelIdle(IdleStateHandler.java:371)
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler$ReaderIdleTimeoutTask.run(IdleStateHandler.java:494)
>         at 
> org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler$AbstractIdleTask.run(IdleStateHandler.java:466)
>         at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.PromiseTask$RunnableAdapter.call(PromiseTask.java:38)
>         at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.ScheduledFutureTask.run(ScheduledFutureTask.java:127)
>         at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
>         at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
>         at 
> org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:333)
>         at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:905)
>         at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>         at java.lang.Thread.run(Thread.java:745){code}
>  
> The configuration of 'hdfs. client. socket time' is reused in the 
> FanOutOneBlockAsyncDFSOutput.class of hbase.
> –
> In the 'FanOutOneBlockAsyncDFSOutput' construction method:
> {code:java}
> FanOutOneBlockAsyncDFSOutput(Configuration conf, FSUtils fsUtils, 
> DistributedFileSystem dfs,
>     DFSClient client, ClientProtocol namenode, String clientName, String src, 
> long fileId,
>     LocatedBlock locatedBlock, Encryptor encryptor, List<Channel> 
> datanodeList,
>     DataChecksum summer, ByteBufAllocator alloc) {
>   this.conf = conf;
>   this.fsUtils = fsUtils;
>   this.dfs = dfs;
>   this.client = client;
>   this.namenode = namenode;
>   this.fileId = fileId;
>   this.clientName = clientName;
>   this.src = src;
>   this.block = locatedBlock.getBlock();
>   this.locations = locatedBlock.getLocations();
>   this.encryptor = encryptor;
>   this.datanodeList = datanodeList;
>   this.summer = summer;
>   this.maxDataLen = MAX_DATA_LEN - (MAX_DATA_LEN % 
> summer.getBytesPerChecksum());
>   this.alloc = alloc;
>   this.buf = alloc.directBuffer(sendBufSizePRedictor.initialSize());
>   this.state = State.STREAMING;
> *  setupReceiver(conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, READ_TIMEOUT));
> } {code}
> ----
> My implementation process:
> 1. add a new configuration in hbase-site.xml
> {code:java}
> + <property>
> + <name>hbase.wal.asyncfsoutput.timeout</name>
> + <value>60000</value>
> + </property> {code}
> 2.modify code
> {code:java}
> 151 + private static final String FANOUT_TIMEOUTKEY = 
> "hbase.wal.asyncfsoutput.timeout";
> 339 - setupReceiver(conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, READ_TIMEOUT));
> 339 + setupReceiver(conf.getInt(FANOUT_TIMEOUTKEY, READ_TIMEOUT));
>  {code}
> 3.repackge
>  
> —
> I would like to consult the community on whether it is possible to design a 
> separate configuration item in FanOutOneBlockAsynchronousDFSOutput.jva to 
> isolate the issue of writing Wal timeout failure due to the small size of 
> dfs.comient.socket time?



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to