-
Notifications
You must be signed in to change notification settings - Fork 476
/
TestBlockManager.java
511 lines (472 loc) · 19.4 KB
/
TestBlockManager.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hdds.scm.block;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.ArrayList;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStore;
import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStoreRDBImpl;
import org.apache.hadoop.hdds.scm.pipeline.MockRatisPipelineProvider;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider;
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager;
import org.apache.hadoop.hdds.scm.server.SCMConfigurator;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
import org.apache.hadoop.ozone.protocol.commands.CreatePipelineCommand;
import org.apache.hadoop.test.GenericTestUtils;
import static org.apache.hadoop.ozone.OzoneConsts.GB;
import static org.apache.hadoop.ozone.OzoneConsts.MB;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
/**
* Tests for SCM Block Manager.
*/
public class TestBlockManager {
private StorageContainerManager scm;
private SCMContainerManager mapping;
private MockNodeManager nodeManager;
private SCMPipelineManager pipelineManager;
private BlockManagerImpl blockManager;
private final static long DEFAULT_BLOCK_SIZE = 128 * MB;
private static HddsProtos.ReplicationFactor factor;
private static HddsProtos.ReplicationType type;
private EventQueue eventQueue;
private int numContainerPerOwnerInPipeline;
private OzoneConfiguration conf;
@Rule
public ExpectedException thrown = ExpectedException.none();
@Rule
public TemporaryFolder folder= new TemporaryFolder();
private SCMMetadataStore scmMetadataStore;
@Before
public void setUp() throws Exception {
conf = SCMTestUtils.getConf();
numContainerPerOwnerInPipeline = conf.getInt(
ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT,
ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT_DEFAULT);
conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, folder.newFolder().toString());
conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false);
conf.setTimeDuration(HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL, 5,
TimeUnit.SECONDS);
// Override the default Node Manager in SCM with this Mock Node Manager.
nodeManager = new MockNodeManager(true, 10);
eventQueue = new EventQueue();
scmMetadataStore = new SCMMetadataStoreRDBImpl(conf);
scmMetadataStore.start(conf);
pipelineManager =
new SCMPipelineManager(conf, nodeManager,
scmMetadataStore.getPipelineTable(),
eventQueue);
pipelineManager.allowPipelineCreation();
PipelineProvider mockRatisProvider =
new MockRatisPipelineProvider(nodeManager,
pipelineManager.getStateManager(), conf, eventQueue);
pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS,
mockRatisProvider);
SCMContainerManager containerManager =
new SCMContainerManager(conf,
scmMetadataStore.getContainerTable(),
scmMetadataStore.getStore(),
pipelineManager);
SCMSafeModeManager safeModeManager = new SCMSafeModeManager(conf,
containerManager.getContainers(), pipelineManager, eventQueue) {
@Override
public void emitSafeModeStatus() {
// skip
}
};
SCMConfigurator configurator = new SCMConfigurator();
configurator.setScmNodeManager(nodeManager);
configurator.setPipelineManager(pipelineManager);
configurator.setContainerManager(containerManager);
configurator.setScmSafeModeManager(safeModeManager);
configurator.setMetadataStore(scmMetadataStore);
scm = TestUtils.getScm(conf, configurator);
// Initialize these fields so that the tests can pass.
mapping = (SCMContainerManager) scm.getContainerManager();
blockManager = (BlockManagerImpl) scm.getScmBlockManager();
DatanodeCommandHandler handler = new DatanodeCommandHandler();
eventQueue.addHandler(SCMEvents.DATANODE_COMMAND, handler);
CloseContainerEventHandler closeContainerHandler =
new CloseContainerEventHandler(pipelineManager, mapping);
eventQueue.addHandler(SCMEvents.CLOSE_CONTAINER, closeContainerHandler);
factor = HddsProtos.ReplicationFactor.THREE;
type = HddsProtos.ReplicationType.RATIS;
blockManager.onMessage(
new SCMSafeModeManager.SafeModeStatus(false, false), null);
}
@After
public void cleanup() throws Exception {
scm.stop();
scm.join();
eventQueue.close();
scmMetadataStore.stop();
}
@Test
public void testAllocateBlock() throws Exception {
pipelineManager.createPipeline(type, factor);
TestUtils.openAllRatisPipelines(pipelineManager);
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor, OzoneConsts.OZONE, new ExcludeList());
Assert.assertNotNull(block);
}
@Test
public void testAllocateBlockWithExclusion() throws Exception {
try {
while (true) {
pipelineManager.createPipeline(type, factor);
}
} catch (IOException e) {
}
TestUtils.openAllRatisPipelines(pipelineManager);
ExcludeList excludeList = new ExcludeList();
excludeList
.addPipeline(pipelineManager.getPipelines(type, factor).get(0).getId());
AllocatedBlock block = blockManager
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, OzoneConsts.OZONE,
excludeList);
Assert.assertNotNull(block);
Assert.assertNotEquals(block.getPipeline().getId(),
excludeList.getPipelineIds().get(0));
for (Pipeline pipeline : pipelineManager.getPipelines(type, factor)) {
excludeList.addPipeline(pipeline.getId());
}
block = blockManager
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, OzoneConsts.OZONE,
excludeList);
Assert.assertNotNull(block);
Assert.assertTrue(
excludeList.getPipelineIds().contains(block.getPipeline().getId()));
}
@Test
public void testAllocateBlockInParallel() throws Exception {
int threadCount = 20;
List<ExecutorService> executors = new ArrayList<>(threadCount);
for (int i = 0; i < threadCount; i++) {
executors.add(Executors.newSingleThreadExecutor());
}
List<CompletableFuture<AllocatedBlock>> futureList =
new ArrayList<>(threadCount);
for (int i = 0; i < threadCount; i++) {
final CompletableFuture<AllocatedBlock> future =
new CompletableFuture<>();
CompletableFuture.supplyAsync(() -> {
try {
future.complete(blockManager
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor,
OzoneConsts.OZONE,
new ExcludeList()));
} catch (IOException e) {
future.completeExceptionally(e);
}
return future;
}, executors.get(i));
futureList.add(future);
}
try {
CompletableFuture
.allOf(futureList.toArray(new CompletableFuture[futureList.size()]))
.get();
} catch (Exception e) {
Assert.fail("testAllocateBlockInParallel failed");
}
}
@Test
public void testBlockDistribution() throws Exception {
int threadCount = numContainerPerOwnerInPipeline *
numContainerPerOwnerInPipeline;
List<ExecutorService> executors = new ArrayList<>(threadCount);
for (int i = 0; i < threadCount; i++) {
executors.add(Executors.newSingleThreadExecutor());
}
pipelineManager.createPipeline(type, factor);
TestUtils.openAllRatisPipelines(pipelineManager);
Map<Long, List<AllocatedBlock>> allocatedBlockMap =
new ConcurrentHashMap<>();
List<CompletableFuture<AllocatedBlock>> futureList =
new ArrayList<>(threadCount);
for (int i = 0; i < threadCount; i++) {
final CompletableFuture<AllocatedBlock> future =
new CompletableFuture<>();
CompletableFuture.supplyAsync(() -> {
try {
List<AllocatedBlock> blockList;
AllocatedBlock block = blockManager
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor,
OzoneConsts.OZONE,
new ExcludeList());
long containerId = block.getBlockID().getContainerID();
if (!allocatedBlockMap.containsKey(containerId)) {
blockList = new ArrayList<>();
} else {
blockList = allocatedBlockMap.get(containerId);
}
blockList.add(block);
allocatedBlockMap.put(containerId, blockList);
future.complete(block);
} catch (IOException e) {
future.completeExceptionally(e);
}
return future;
}, executors.get(i));
futureList.add(future);
}
try {
CompletableFuture
.allOf(futureList.toArray(
new CompletableFuture[futureList.size()])).get();
Assert.assertTrue(pipelineManager.getPipelines(type).size() == 1);
Assert.assertTrue(
allocatedBlockMap.size() == numContainerPerOwnerInPipeline);
Assert.assertTrue(allocatedBlockMap.
values().size() == numContainerPerOwnerInPipeline);
allocatedBlockMap.values().stream().forEach(v -> {
Assert.assertTrue(v.size() == numContainerPerOwnerInPipeline);
});
} catch (Exception e) {
Assert.fail("testAllocateBlockInParallel failed");
}
}
@Test
public void testBlockDistributionWithMultipleDisks() throws Exception {
int threadCount = numContainerPerOwnerInPipeline *
numContainerPerOwnerInPipeline;
nodeManager.setNumHealthyVolumes(numContainerPerOwnerInPipeline);
List<ExecutorService> executors = new ArrayList<>(threadCount);
for (int i = 0; i < threadCount; i++) {
executors.add(Executors.newSingleThreadExecutor());
}
pipelineManager.createPipeline(type, factor);
TestUtils.openAllRatisPipelines(pipelineManager);
Map<Long, List<AllocatedBlock>> allocatedBlockMap =
new ConcurrentHashMap<>();
List<CompletableFuture<AllocatedBlock>> futureList =
new ArrayList<>(threadCount);
for (int i = 0; i < threadCount; i++) {
final CompletableFuture<AllocatedBlock> future =
new CompletableFuture<>();
CompletableFuture.supplyAsync(() -> {
try {
List<AllocatedBlock> blockList;
AllocatedBlock block = blockManager
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor,
OzoneConsts.OZONE,
new ExcludeList());
long containerId = block.getBlockID().getContainerID();
if (!allocatedBlockMap.containsKey(containerId)) {
blockList = new ArrayList<>();
} else {
blockList = allocatedBlockMap.get(containerId);
}
blockList.add(block);
allocatedBlockMap.put(containerId, blockList);
future.complete(block);
} catch (IOException e) {
future.completeExceptionally(e);
}
return future;
}, executors.get(i));
futureList.add(future);
}
try {
CompletableFuture
.allOf(futureList.toArray(
new CompletableFuture[futureList.size()])).get();
Assert.assertTrue(
pipelineManager.getPipelines(type).size() == 1);
Pipeline pipeline = pipelineManager.getPipelines(type).get(0);
// total no of containers to be created will be number of healthy
// volumes * number of numContainerPerOwnerInPipeline which is equal to
// the thread count
Assert.assertTrue(threadCount == pipelineManager.
getNumberOfContainers(pipeline.getId()));
Assert.assertTrue(
allocatedBlockMap.size() == threadCount);
Assert.assertTrue(allocatedBlockMap.
values().size() == threadCount);
allocatedBlockMap.values().stream().forEach(v -> {
Assert.assertTrue(v.size() == 1);
});
} catch (Exception e) {
Assert.fail("testAllocateBlockInParallel failed");
}
}
@Test
public void testAllocateOversizedBlock() throws Exception {
long size = 6 * GB;
thrown.expectMessage("Unsupported block size");
AllocatedBlock block = blockManager.allocateBlock(size,
type, factor, OzoneConsts.OZONE, new ExcludeList());
}
@Test
public void testAllocateBlockFailureInSafeMode() throws Exception {
blockManager.onMessage(
new SCMSafeModeManager.SafeModeStatus(true, true), null);
// Test1: In safe mode expect an SCMException.
thrown.expectMessage("SafeModePrecheck failed for "
+ "allocateBlock");
blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor, OzoneConsts.OZONE, new ExcludeList());
}
@Test
public void testAllocateBlockSucInSafeMode() throws Exception {
// Test2: Exit safe mode and then try allocateBock again.
Assert.assertNotNull(blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor, OzoneConsts.OZONE, new ExcludeList()));
}
@Test(timeout = 10000)
public void testMultipleBlockAllocation()
throws IOException, TimeoutException, InterruptedException {
pipelineManager.createPipeline(type, factor);
pipelineManager.createPipeline(type, factor);
TestUtils.openAllRatisPipelines(pipelineManager);
AllocatedBlock allocatedBlock = blockManager
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, OzoneConsts.OZONE,
new ExcludeList());
// block should be allocated in different pipelines
GenericTestUtils.waitFor(() -> {
try {
AllocatedBlock block = blockManager
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, OzoneConsts.OZONE,
new ExcludeList());
return !block.getPipeline().getId()
.equals(allocatedBlock.getPipeline().getId());
} catch (IOException e) {
}
return false;
}, 100, 1000);
}
private boolean verifyNumberOfContainersInPipelines(
int numContainersPerPipeline) {
try {
for (Pipeline pipeline : pipelineManager.getPipelines(type, factor)) {
if (pipelineManager.getNumberOfContainers(pipeline.getId())
!= numContainersPerPipeline) {
return false;
}
}
} catch (IOException e) {
return false;
}
return true;
}
@Test(timeout = 10000)
public void testMultipleBlockAllocationWithClosedContainer()
throws IOException, TimeoutException, InterruptedException {
// create pipelines
for (int i = 0;
i < nodeManager.getNodes(HddsProtos.NodeState.HEALTHY).size() / factor
.getNumber(); i++) {
pipelineManager.createPipeline(type, factor);
}
TestUtils.openAllRatisPipelines(pipelineManager);
// wait till each pipeline has the configured number of containers.
// After this each pipeline has numContainerPerOwnerInPipeline containers
// for each owner
GenericTestUtils.waitFor(() -> {
try {
blockManager
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, OzoneConsts.OZONE,
new ExcludeList());
} catch (IOException e) {
}
return verifyNumberOfContainersInPipelines(
numContainerPerOwnerInPipeline);
}, 10, 1000);
// close all the containers in all the pipelines
for (Pipeline pipeline : pipelineManager.getPipelines(type, factor)) {
for (ContainerID cid : pipelineManager
.getContainersInPipeline(pipeline.getId())) {
eventQueue.fireEvent(SCMEvents.CLOSE_CONTAINER, cid);
}
}
// wait till no containers are left in the pipelines
GenericTestUtils
.waitFor(() -> verifyNumberOfContainersInPipelines(0), 10, 5000);
// allocate block so that each pipeline has the configured number of
// containers.
GenericTestUtils.waitFor(() -> {
try {
blockManager
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, OzoneConsts.OZONE,
new ExcludeList());
} catch (IOException e) {
}
return verifyNumberOfContainersInPipelines(
numContainerPerOwnerInPipeline);
}, 10, 1000);
}
@Test(timeout = 10000)
public void testBlockAllocationWithNoAvailablePipelines()
throws IOException, TimeoutException, InterruptedException {
for (Pipeline pipeline : pipelineManager.getPipelines()) {
pipelineManager.finalizeAndDestroyPipeline(pipeline, false);
}
Assert.assertEquals(0, pipelineManager.getPipelines(type, factor).size());
Assert.assertNotNull(blockManager
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, OzoneConsts.OZONE,
new ExcludeList()));
}
private class DatanodeCommandHandler implements
EventHandler<CommandForDatanode> {
@Override
public void onMessage(final CommandForDatanode command,
final EventPublisher publisher) {
final SCMCommandProto.Type commandType = command.getCommand().getType();
if (commandType == SCMCommandProto.Type.createPipelineCommand) {
CreatePipelineCommand createCommand =
(CreatePipelineCommand) command.getCommand();
try {
pipelineManager.openPipeline(createCommand.getPipelineID());
} catch (IOException e) {
}
}
}
}
}