Skip to content

Commit ca6f5af

Browse files
authored
HDFS-16942. Send error to datanode if FBR is rejected due to bad lease (#5460)
1 parent 734f7ab commit ca6f5af

File tree

7 files changed

+154
-3
lines changed

7 files changed

+154
-3
lines changed

hadoop-client-modules/hadoop-client-api/pom.xml

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -126,6 +126,12 @@
126126
<exclude>org/apache/hadoop/yarn/client/api/package-info.class</exclude>
127127
</excludes>
128128
</filter>
129+
<filter>
130+
<artifact>org.apache.hadoop:*</artifact>
131+
<excludes>
132+
<exclude>org/apache/hadoop/hdfs/server/protocol/package-info.class</exclude>
133+
</excludes>
134+
</filter>
129135
</filters>
130136
<relocations>
131137
<relocation>

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -63,6 +63,7 @@
6363
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
6464
import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
6565
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
66+
import org.apache.hadoop.hdfs.server.protocol.InvalidBlockReportLeaseException;
6667
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
6768
import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
6869
import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports;
@@ -791,6 +792,9 @@ private void offerService() throws Exception {
791792
shouldServiceRun = false;
792793
return;
793794
}
795+
if (InvalidBlockReportLeaseException.class.getName().equals(reClass)) {
796+
fullBlockReportLeaseId = 0;
797+
}
794798
LOG.warn("RemoteException in offerService", re);
795799
sleepAfterException();
796800
} catch (IOException e) {

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -172,6 +172,7 @@
172172
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
173173
import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
174174
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
175+
import org.apache.hadoop.hdfs.server.protocol.InvalidBlockReportLeaseException;
175176
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
176177
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
177178
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
@@ -1651,6 +1652,8 @@ public DatanodeCommand blockReport(final DatanodeRegistration nodeReg,
16511652
bm.processReport(nodeReg, reports[index].getStorage(),
16521653
blocks, context));
16531654
}
1655+
} else {
1656+
throw new InvalidBlockReportLeaseException(context.getReportId(), context.getLeaseId());
16541657
}
16551658
} catch (UnregisteredNodeException une) {
16561659
LOG.warn("Datanode {} is attempting to report but not register yet.",
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,41 @@
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+
19+
package org.apache.hadoop.hdfs.server.protocol;
20+
21+
import java.io.IOException;
22+
23+
import org.apache.hadoop.classification.InterfaceAudience;
24+
import org.apache.hadoop.classification.InterfaceStability;
25+
26+
/**
27+
* This exception is thrown when a datanode sends a full block report but it is
28+
* rejected by the Namenode due to an invalid lease (expired or otherwise).
29+
*
30+
*/
31+
@InterfaceAudience.Private
32+
@InterfaceStability.Evolving
33+
public class InvalidBlockReportLeaseException extends IOException {
34+
/** for java.io.Serializable. */
35+
private static final long serialVersionUID = 1L;
36+
37+
public InvalidBlockReportLeaseException(long blockReportID, long leaseID) {
38+
super("Block report 0x" + Long.toHexString(blockReportID) + " was rejected as lease 0x"
39+
+ Long.toHexString(leaseID) + " is invalid");
40+
}
41+
}
Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,27 @@
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+
19+
/**
20+
* This package provides classes for the namenode server protocol.
21+
*/
22+
@InterfaceAudience.Private
23+
@InterfaceStability.Evolving
24+
package org.apache.hadoop.hdfs.server.protocol;
25+
26+
import org.apache.hadoop.classification.InterfaceAudience;
27+
import org.apache.hadoop.classification.InterfaceStability;

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockReportLease.java

Lines changed: 69 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
3030
import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
3131
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
32+
import org.apache.hadoop.hdfs.server.protocol.InvalidBlockReportLeaseException;
3233
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
3334
import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
3435
import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
@@ -41,12 +42,14 @@
4142
import java.util.ArrayList;
4243
import java.util.List;
4344
import java.util.Random;
45+
import java.util.concurrent.ExecutionException;
4446
import java.util.concurrent.ExecutorService;
4547
import java.util.concurrent.Executors;
4648
import java.util.concurrent.Future;
4749

4850
import static org.junit.Assert.assertEquals;
4951
import static org.junit.Assert.assertTrue;
52+
import static org.junit.jupiter.api.Assertions.assertNotNull;
5053
import static org.mockito.ArgumentMatchers.any;
5154
import static org.mockito.Mockito.doAnswer;
5255
import static org.mockito.Mockito.spy;
@@ -137,6 +140,72 @@ public void testCheckBlockReportLease() throws Exception {
137140
}
138141
}
139142

143+
@Test
144+
public void testExceptionThrownWhenFBRLeaseExpired() throws Exception {
145+
HdfsConfiguration conf = new HdfsConfiguration();
146+
Random rand = new Random();
147+
148+
try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
149+
.numDataNodes(1).build()) {
150+
cluster.waitActive();
151+
152+
FSNamesystem fsn = cluster.getNamesystem();
153+
BlockManager blockManager = fsn.getBlockManager();
154+
BlockManager spyBlockManager = spy(blockManager);
155+
fsn.setBlockManagerForTesting(spyBlockManager);
156+
String poolId = cluster.getNamesystem().getBlockPoolId();
157+
158+
NamenodeProtocols rpcServer = cluster.getNameNodeRpc();
159+
160+
// Test based on one DataNode report to Namenode
161+
DataNode dn = cluster.getDataNodes().get(0);
162+
DatanodeDescriptor datanodeDescriptor = spyBlockManager
163+
.getDatanodeManager().getDatanode(dn.getDatanodeId());
164+
165+
DatanodeRegistration dnRegistration = dn.getDNRegistrationForBP(poolId);
166+
StorageReport[] storages = dn.getFSDataset().getStorageReports(poolId);
167+
168+
// Send heartbeat and request full block report lease
169+
HeartbeatResponse hbResponse = rpcServer.sendHeartbeat(
170+
dnRegistration, storages, 0, 0, 0, 0, 0, null, true,
171+
SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT);
172+
173+
// Remove full block report lease about dn
174+
spyBlockManager.getBlockReportLeaseManager()
175+
.removeLease(datanodeDescriptor);
176+
177+
ExecutorService pool = Executors.newFixedThreadPool(1);
178+
179+
// Trigger sendBlockReport
180+
BlockReportContext brContext = new BlockReportContext(1, 0,
181+
rand.nextLong(), hbResponse.getFullBlockReportLeaseId());
182+
Future<DatanodeCommand> sendBRfuturea = pool.submit(() -> {
183+
// Build every storage with 100 blocks for sending report
184+
DatanodeStorage[] datanodeStorages
185+
= new DatanodeStorage[storages.length];
186+
for (int i = 0; i < storages.length; i++) {
187+
datanodeStorages[i] = storages[i].getStorage();
188+
}
189+
StorageBlockReport[] reports = createReports(datanodeStorages, 100);
190+
191+
// Send blockReport
192+
return rpcServer.blockReport(dnRegistration, poolId, reports,
193+
brContext);
194+
});
195+
196+
// Get result, it will not null if process successfully
197+
ExecutionException exception = null;
198+
try {
199+
sendBRfuturea.get();
200+
} catch (ExecutionException e) {
201+
exception = e;
202+
}
203+
assertNotNull(exception);
204+
assertEquals(InvalidBlockReportLeaseException.class,
205+
exception.getCause().getClass());
206+
}
207+
}
208+
140209
@Test
141210
public void testCheckBlockReportLeaseWhenDnUnregister() throws Exception {
142211
HdfsConfiguration conf = new HdfsConfiguration();

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import org.apache.hadoop.fs.StorageType;
2525
import org.apache.hadoop.hdfs.HdfsConfiguration;
2626
import org.apache.hadoop.hdfs.MiniDFSCluster;
27+
import org.apache.hadoop.hdfs.server.protocol.InvalidBlockReportLeaseException;
2728
import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
2829

2930
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
@@ -39,7 +40,6 @@
3940

4041
import java.io.File;
4142
import java.io.IOException;
42-
import java.net.ConnectException;
4343
import java.net.InetSocketAddress;
4444
import java.util.ArrayList;
4545
import java.util.Collections;
@@ -1187,8 +1187,9 @@ public Object answer(InvocationOnMock invocation)
11871187
// just reject and wait until DN request for a new leaseId
11881188
if(leaseId == 1) {
11891189
firstLeaseId = leaseId;
1190-
throw new ConnectException(
1191-
"network is not reachable for test. ");
1190+
InvalidBlockReportLeaseException e =
1191+
new InvalidBlockReportLeaseException(context.getReportId(), 1);
1192+
throw new RemoteException(e.getClass().getName(), e.getMessage());
11921193
} else {
11931194
secondLeaseId = leaseId;
11941195
return null;

0 commit comments

Comments
 (0)