Skip to content

Commit 4610c75

Browse files
comnetworkApache9
authored andcommitted
HBASE-26679 Wait on the future returned by FanOutOneBlockAsyncDFSOutput.flush would stuck (#4039)
Signed-off-by: Duo Zhang <zhangduo@apache.org>
1 parent 9f1f705 commit 4610c75

File tree

2 files changed

+266
-2
lines changed

2 files changed

+266
-2
lines changed

hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java

Lines changed: 17 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.WRITER_IDLE;
2727
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
2828

29+
import com.google.errorprone.annotations.RestrictedApi;
2930
import java.io.IOException;
3031
import java.io.InterruptedIOException;
3132
import java.nio.ByteBuffer;
@@ -232,7 +233,11 @@ private void completed(Channel channel) {
232233
// so that the implementation will not burn up our brain as there are multiple state changes and
233234
// checks.
234235
private synchronized void failed(Channel channel, Supplier<Throwable> errorSupplier) {
235-
if (state == State.BROKEN || state == State.CLOSED) {
236+
if (state == State.CLOSED) {
237+
return;
238+
}
239+
if (state == State.BROKEN) {
240+
failWaitingAckQueue(channel, errorSupplier);
236241
return;
237242
}
238243
if (state == State.CLOSING) {
@@ -244,6 +249,11 @@ private synchronized void failed(Channel channel, Supplier<Throwable> errorSuppl
244249
}
245250
// disable further write, and fail all pending ack.
246251
state = State.BROKEN;
252+
failWaitingAckQueue(channel, errorSupplier);
253+
datanodeInfoMap.keySet().forEach(ChannelOutboundInvoker::close);
254+
}
255+
256+
private void failWaitingAckQueue(Channel channel, Supplier<Throwable> errorSupplier) {
247257
Throwable error = errorSupplier.get();
248258
for (Iterator<Callback> iter = waitingAckQueue.iterator(); iter.hasNext();) {
249259
Callback c = iter.next();
@@ -260,7 +270,6 @@ private synchronized void failed(Channel channel, Supplier<Throwable> errorSuppl
260270
}
261271
break;
262272
}
263-
datanodeInfoMap.keySet().forEach(ChannelOutboundInvoker::close);
264273
}
265274

266275
@Sharable
@@ -599,4 +608,10 @@ public boolean isBroken() {
599608
public long getSyncedLength() {
600609
return this.ackedBlockLength;
601610
}
611+
612+
@RestrictedApi(explanation = "Should only be called in tests", link = "",
613+
allowedOnPath = ".*/src/test/.*")
614+
Map<Channel, DatanodeInfo> getDatanodeInfoMap() {
615+
return this.datanodeInfoMap;
616+
}
602617
}
Lines changed: 249 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,249 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.hbase.io.asyncfs;
19+
20+
21+
import static org.junit.Assert.assertTrue;
22+
import static org.junit.Assert.fail;
23+
24+
import java.io.IOException;
25+
import java.util.ArrayList;
26+
import java.util.Iterator;
27+
import java.util.List;
28+
import java.util.Map;
29+
import java.util.concurrent.CompletableFuture;
30+
import java.util.concurrent.CyclicBarrier;
31+
import java.util.concurrent.ExecutionException;
32+
import java.util.concurrent.ThreadLocalRandom;
33+
import org.apache.hadoop.fs.Path;
34+
import org.apache.hadoop.hbase.HBaseClassTestRule;
35+
import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor;
36+
import org.apache.hadoop.hbase.testclassification.MediumTests;
37+
import org.apache.hadoop.hbase.testclassification.MiscTests;
38+
import org.apache.hadoop.hdfs.DistributedFileSystem;
39+
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
40+
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
41+
import org.apache.hadoop.hdfs.server.datanode.DataNode;
42+
import org.junit.AfterClass;
43+
import org.junit.BeforeClass;
44+
import org.junit.ClassRule;
45+
import org.junit.Rule;
46+
import org.junit.Test;
47+
import org.junit.experimental.categories.Category;
48+
import org.junit.rules.TestName;
49+
import org.slf4j.Logger;
50+
import org.slf4j.LoggerFactory;
51+
import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
52+
import org.apache.hbase.thirdparty.io.netty.channel.Channel;
53+
import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
54+
import org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter;
55+
import org.apache.hbase.thirdparty.io.netty.channel.EventLoop;
56+
import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
57+
import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
58+
import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
59+
60+
61+
/**
62+
* Testcase for HBASE-26679, here we introduce a separate test class and not put the testcase in
63+
* {@link TestFanOutOneBlockAsyncDFSOutput} because we will send heartbeat to DN when there is no
64+
* out going packet, the timeout is controlled by
65+
* {@link TestFanOutOneBlockAsyncDFSOutput#READ_TIMEOUT_MS},which is 2 seconds, it will keep sending
66+
* package out and DN will respond immedately and then mess up the testing handler added by us. So
67+
* in this test class we use the default value for timeout which is 60 seconds and it is enough for
68+
* this test.
69+
*/
70+
@Category({ MiscTests.class, MediumTests.class })
71+
public class TestFanOutOneBlockAsyncDFSOutputHang extends AsyncFSTestBase {
72+
73+
@ClassRule
74+
public static final HBaseClassTestRule CLASS_RULE =
75+
HBaseClassTestRule.forClass(TestFanOutOneBlockAsyncDFSOutputHang.class);
76+
77+
private static final Logger LOG =
78+
LoggerFactory.getLogger(TestFanOutOneBlockAsyncDFSOutputHang.class);
79+
80+
private static DistributedFileSystem FS;
81+
82+
private static EventLoopGroup EVENT_LOOP_GROUP;
83+
84+
private static Class<? extends Channel> CHANNEL_CLASS;
85+
86+
private static StreamSlowMonitor MONITOR;
87+
88+
private static FanOutOneBlockAsyncDFSOutput OUT;
89+
90+
@Rule
91+
public TestName name = new TestName();
92+
93+
@BeforeClass
94+
public static void setUp() throws Exception {
95+
startMiniDFSCluster(2);
96+
FS = CLUSTER.getFileSystem();
97+
EVENT_LOOP_GROUP = new NioEventLoopGroup();
98+
CHANNEL_CLASS = NioSocketChannel.class;
99+
MONITOR = StreamSlowMonitor.create(UTIL.getConfiguration(), "testMonitor");
100+
Path f = new Path("/testHang");
101+
EventLoop eventLoop = EVENT_LOOP_GROUP.next();
102+
OUT = FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 2,
103+
FS.getDefaultBlockSize(), eventLoop, CHANNEL_CLASS, MONITOR);
104+
}
105+
106+
@AfterClass
107+
public static void tearDown() throws IOException, InterruptedException {
108+
if (OUT != null) {
109+
OUT.recoverAndClose(null);
110+
}
111+
if (EVENT_LOOP_GROUP != null) {
112+
EVENT_LOOP_GROUP.shutdownGracefully().sync();
113+
}
114+
shutdownMiniDFSCluster();
115+
}
116+
117+
/**
118+
* <pre>
119+
* This test is for HBASE-26679. Consider there are two dataNodes: dn1 and dn2,dn2 is a slow DN.
120+
* The threads sequence before HBASE-26679 is:
121+
* 1.We write some data to {@link FanOutOneBlockAsyncDFSOutput} and then flush it, there are one
122+
* {@link FanOutOneBlockAsyncDFSOutput.Callback} in
123+
* {@link FanOutOneBlockAsyncDFSOutput#waitingAckQueue}.
124+
* 2.The ack from dn1 arrives firstly and triggers Netty to invoke
125+
* {@link FanOutOneBlockAsyncDFSOutput#completed} with dn1's channel, then in
126+
* {@link FanOutOneBlockAsyncDFSOutput#completed}, dn1's channel is removed from
127+
* {@link FanOutOneBlockAsyncDFSOutput.Callback#unfinishedReplicas}.
128+
* 3.But dn2 responds slowly, before dn2 sending ack,dn1 is shut down or have a exception,
129+
* so {@link FanOutOneBlockAsyncDFSOutput#failed} is triggered by Netty with dn1's channel,
130+
* and because the {@link FanOutOneBlockAsyncDFSOutput.Callback#unfinishedReplicas} does not
131+
* contain dn1's channel,the {@link FanOutOneBlockAsyncDFSOutput.Callback} is skipped in
132+
* {@link FanOutOneBlockAsyncDFSOutput#failed} method,and
133+
* {@link FanOutOneBlockAsyncDFSOutput#state} is set to
134+
* {@link FanOutOneBlockAsyncDFSOutput.State#BROKEN},and dn1,dn2 are all closed at the end of
135+
* {@link FanOutOneBlockAsyncDFSOutput#failed}.
136+
* 4.{@link FanOutOneBlockAsyncDFSOutput#failed} is triggered again by dn2 because it is closed,
137+
* but because {@link FanOutOneBlockAsyncDFSOutput#state} is already
138+
* {@link FanOutOneBlockAsyncDFSOutput.State#BROKEN},the whole
139+
* {@link FanOutOneBlockAsyncDFSOutput#failed} is skipped. So wait on the future
140+
* returned by {@link FanOutOneBlockAsyncDFSOutput#flush} would be stuck for ever.
141+
* After HBASE-26679, for above step 4,even if the {@link FanOutOneBlockAsyncDFSOutput#state}
142+
* is already {@link FanOutOneBlockAsyncDFSOutput.State#BROKEN}, we would still try to trigger
143+
* {@link FanOutOneBlockAsyncDFSOutput.Callback#future}.
144+
* </pre>
145+
*/
146+
@Test
147+
public void testFlushHangWhenOneDataNodeFailedBeforeOtherDataNodeAck() throws Exception {
148+
149+
DataNodeProperties firstDataNodeProperties = null;
150+
try {
151+
152+
final CyclicBarrier dn1AckReceivedCyclicBarrier = new CyclicBarrier(2);
153+
Map<Channel, DatanodeInfo> datanodeInfoMap = OUT.getDatanodeInfoMap();
154+
Iterator<Map.Entry<Channel, DatanodeInfo>> iterator = datanodeInfoMap.entrySet().iterator();
155+
assertTrue(iterator.hasNext());
156+
Map.Entry<Channel, DatanodeInfo> dn1Entry = iterator.next();
157+
Channel dn1Channel = dn1Entry.getKey();
158+
DatanodeInfo dn1DatanodeInfo = dn1Entry.getValue();
159+
final List<String> protobufDecoderNames = new ArrayList<String>();
160+
dn1Channel.pipeline().forEach((entry) -> {
161+
if (ProtobufDecoder.class.isInstance(entry.getValue())) {
162+
protobufDecoderNames.add(entry.getKey());
163+
}
164+
});
165+
assertTrue(protobufDecoderNames.size() == 1);
166+
dn1Channel.pipeline().addAfter(protobufDecoderNames.get(0), "dn1AckReceivedHandler",
167+
new ChannelInboundHandlerAdapter() {
168+
@Override
169+
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
170+
super.channelRead(ctx, msg);
171+
dn1AckReceivedCyclicBarrier.await();
172+
}
173+
});
174+
175+
assertTrue(iterator.hasNext());
176+
Map.Entry<Channel, DatanodeInfo> dn2Entry = iterator.next();
177+
Channel dn2Channel = dn2Entry.getKey();
178+
179+
/**
180+
* Here we add a {@link ChannelInboundHandlerAdapter} to eat all the responses to simulate a
181+
* slow dn2.
182+
*/
183+
dn2Channel.pipeline().addFirst(new ChannelInboundHandlerAdapter() {
184+
185+
@Override
186+
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
187+
if (!(msg instanceof ByteBuf)) {
188+
ctx.fireChannelRead(msg);
189+
}
190+
}
191+
});
192+
193+
byte[] b = new byte[10];
194+
ThreadLocalRandom.current().nextBytes(b);
195+
OUT.write(b, 0, b.length);
196+
CompletableFuture<Long> future = OUT.flush(false);
197+
/**
198+
* Wait for ack from dn1.
199+
*/
200+
dn1AckReceivedCyclicBarrier.await();
201+
/**
202+
* First ack is received from dn1,we could stop dn1 now.
203+
*/
204+
firstDataNodeProperties = findAndKillFirstDataNode(dn1DatanodeInfo);
205+
assertTrue(firstDataNodeProperties != null);
206+
try {
207+
/**
208+
* Before HBASE-26679,here we should be stuck, after HBASE-26679,we would fail soon with
209+
* {@link ExecutionException}.
210+
*/
211+
future.get();
212+
fail();
213+
} catch (ExecutionException e) {
214+
assertTrue(e != null);
215+
LOG.info("expected exception caught when get future", e);
216+
}
217+
/**
218+
* Make sure all the data node channel are closed.
219+
*/
220+
datanodeInfoMap.keySet().forEach(ch -> {
221+
try {
222+
ch.closeFuture().get();
223+
} catch (InterruptedException | ExecutionException e) {
224+
throw new RuntimeException(e);
225+
}
226+
});
227+
} finally {
228+
if (firstDataNodeProperties != null) {
229+
CLUSTER.restartDataNode(firstDataNodeProperties);
230+
}
231+
}
232+
}
233+
234+
private static DataNodeProperties findAndKillFirstDataNode(DatanodeInfo firstDatanodeInfo) {
235+
assertTrue(firstDatanodeInfo != null);
236+
ArrayList<DataNode> dataNodes = CLUSTER.getDataNodes();
237+
ArrayList<Integer> foundIndexes = new ArrayList<Integer>();
238+
int index = 0;
239+
for (DataNode dataNode : dataNodes) {
240+
if (firstDatanodeInfo.getXferAddr().equals(dataNode.getDatanodeId().getXferAddr())) {
241+
foundIndexes.add(index);
242+
}
243+
index++;
244+
}
245+
assertTrue(foundIndexes.size() == 1);
246+
return CLUSTER.stopDataNode(foundIndexes.get(0));
247+
}
248+
249+
}

0 commit comments

Comments
 (0)