forked from apache/ozone
-
Notifications
You must be signed in to change notification settings - Fork 0
/
SCMContainerManager.java
613 lines (561 loc) · 20.6 KB
/
SCMContainerManager.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
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
/*
* 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>
* <p>http://www.apache.org/licenses/LICENSE-2.0
* <p>
* <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.container;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.metrics.SCMContainerManagerMetrics;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
import org.apache.hadoop.hdds.utils.db.BatchOperationHandler;
import org.apache.hadoop.hdds.utils.db.Table;
import org.apache.hadoop.hdds.utils.db.Table.KeyValue;
import org.apache.hadoop.hdds.utils.db.TableIterator;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* ContainerManager class contains the mapping from a name to a pipeline
* mapping. This is used by SCM when allocating new locations and when
* looking up a key.
*/
public class SCMContainerManager implements ContainerManager {
private static final Logger LOG = LoggerFactory.getLogger(
SCMContainerManager.class);
private final Lock lock;
private final PipelineManager pipelineManager;
private final ContainerStateManager containerStateManager;
private final int numContainerPerVolume;
private final SCMContainerManagerMetrics scmContainerManagerMetrics;
private Table<ContainerID, ContainerInfo> containerStore;
private BatchOperationHandler batchHandler;
/**
* Constructs a mapping class that creates mapping between container names
* and pipelines.
* <p>
* passed to LevelDB and this memory is allocated in Native code space.
* CacheSize is specified
* in MB.
*
* @param conf - {@link ConfigurationSource}
* @param pipelineManager - {@link PipelineManager}
* @throws IOException on Failure.
*/
public SCMContainerManager(
final ConfigurationSource conf,
Table<ContainerID, ContainerInfo> containerStore,
BatchOperationHandler batchHandler,
PipelineManager pipelineManager)
throws IOException {
this.batchHandler = batchHandler;
this.containerStore = containerStore;
this.lock = new ReentrantLock();
this.pipelineManager = pipelineManager;
this.containerStateManager = new ContainerStateManager(conf);
this.numContainerPerVolume = conf
.getInt(ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT,
ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT_DEFAULT);
loadExistingContainers();
scmContainerManagerMetrics = SCMContainerManagerMetrics.create();
}
private void loadExistingContainers() throws IOException {
TableIterator<ContainerID, ? extends KeyValue<ContainerID, ContainerInfo>>
iterator = containerStore.iterator();
while (iterator.hasNext()) {
ContainerInfo container = iterator.next().getValue();
Preconditions.checkNotNull(container);
containerStateManager.loadContainer(container);
try {
if (container.getState() == LifeCycleState.OPEN) {
pipelineManager.addContainerToPipeline(container.getPipelineID(),
ContainerID.valueof(container.getContainerID()));
}
} catch (PipelineNotFoundException ex) {
LOG.warn("Found a Container {} which is in {} state with pipeline {} " +
"that does not exist. Closing Container.", container,
container.getState(), container.getPipelineID());
updateContainerState(container.containerID(),
HddsProtos.LifeCycleEvent.FINALIZE, true);
}
}
}
@VisibleForTesting
// TODO: remove this later.
public ContainerStateManager getContainerStateManager() {
return containerStateManager;
}
@Override
public Set<ContainerID> getContainerIDs() {
lock.lock();
try {
return containerStateManager.getAllContainerIDs();
} finally {
lock.unlock();
}
}
@Override
public List<ContainerInfo> getContainers() {
lock.lock();
try {
return containerStateManager.getAllContainerIDs().stream().map(id -> {
try {
return containerStateManager.getContainer(id);
} catch (ContainerNotFoundException e) {
// How can this happen?
return null;
}
}).filter(Objects::nonNull).collect(Collectors.toList());
} finally {
lock.unlock();
}
}
@Override
public List<ContainerInfo> getContainers(LifeCycleState state) {
lock.lock();
try {
return containerStateManager.getContainerIDsByState(state).stream()
.map(id -> {
try {
return containerStateManager.getContainer(id);
} catch (ContainerNotFoundException e) {
// How can this happen?
return null;
}
}).filter(Objects::nonNull).collect(Collectors.toList());
} finally {
lock.unlock();
}
}
/**
* Get number of containers in the given state.
*
* @param state {@link LifeCycleState}
* @return Count
*/
public Integer getContainerCountByState(LifeCycleState state) {
return containerStateManager.getContainerCountByState(state);
}
/**
* {@inheritDoc}
*/
@Override
public ContainerInfo getContainer(final ContainerID containerID)
throws ContainerNotFoundException {
return containerStateManager.getContainer(containerID);
}
@Override
public boolean exists(ContainerID containerID) {
lock.lock();
try {
return (containerStateManager.getContainer(containerID) != null);
} catch (ContainerNotFoundException e) {
return false;
} finally {
lock.unlock();
}
}
/**
* {@inheritDoc}
*/
@Override
public List<ContainerInfo> listContainer(ContainerID startContainerID,
int count) {
lock.lock();
try {
scmContainerManagerMetrics.incNumListContainersOps();
final long startId = startContainerID == null ?
0 : startContainerID.getId();
final List<ContainerID> containersIds =
new ArrayList<>(containerStateManager.getAllContainerIDs());
Collections.sort(containersIds);
return containersIds.stream()
.filter(id -> id.getId() > startId)
.limit(count)
.map(id -> {
try {
return containerStateManager.getContainer(id);
} catch (ContainerNotFoundException ex) {
// This can never happen, as we hold lock no one else can remove
// the container after we got the container ids.
LOG.warn("Container Missing.", ex);
return null;
}
}).collect(Collectors.toList());
} finally {
lock.unlock();
}
}
/**
* Allocates a new container.
*
* @param replicationFactor - replication factor of the container.
* @param owner - The string name of the Service that owns this container.
* @return - Pipeline that makes up this container.
* @throws IOException - Exception
*/
@Override
public ContainerInfo allocateContainer(final ReplicationType type,
final ReplicationFactor replicationFactor, final String owner)
throws IOException {
try {
lock.lock();
ContainerInfo containerInfo = null;
try {
containerInfo =
containerStateManager.allocateContainer(pipelineManager, type,
replicationFactor, owner);
} catch (IOException ex) {
scmContainerManagerMetrics.incNumFailureCreateContainers();
throw ex;
}
// Add container to DB.
try {
addContainerToDB(containerInfo);
} catch (IOException ex) {
// When adding to DB failed, we are removing from containerStateMap.
// We should also remove from pipeline2Container Map in
// PipelineStateManager.
pipelineManager.removeContainerFromPipeline(
containerInfo.getPipelineID(),
new ContainerID(containerInfo.getContainerID()));
throw ex;
}
return containerInfo;
} finally {
lock.unlock();
}
}
/**
* Deletes a container from SCM.
*
* @param containerID - Container ID
* @throws IOException if container doesn't exist or container store failed
* to delete the
* specified key.
*/
@Override
public void deleteContainer(ContainerID containerID) throws IOException {
lock.lock();
try {
containerStateManager.removeContainer(containerID);
if (containerStore.get(containerID) != null) {
containerStore.delete(containerID);
} else {
// Where did the container go? o_O
LOG.warn("Unable to remove the container {} from container store," +
" it's missing!", containerID);
}
scmContainerManagerMetrics.incNumSuccessfulDeleteContainers();
} catch (ContainerNotFoundException cnfe) {
scmContainerManagerMetrics.incNumFailureDeleteContainers();
throw new SCMException(
"Failed to delete container " + containerID + ", reason : " +
"container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
} finally {
lock.unlock();
}
}
/**
* {@inheritDoc} Used by client to update container state on SCM.
*/
@Override
public HddsProtos.LifeCycleState updateContainerState(
ContainerID containerID, HddsProtos.LifeCycleEvent event)
throws IOException {
// Should we return the updated ContainerInfo instead of LifeCycleState?
return updateContainerState(containerID, event, false);
}
private HddsProtos.LifeCycleState updateContainerState(
ContainerID containerID, HddsProtos.LifeCycleEvent event,
boolean skipPipelineToContainerRemove)
throws IOException {
// Should we return the updated ContainerInfo instead of LifeCycleState?
lock.lock();
try {
final ContainerInfo container = containerStateManager
.getContainer(containerID);
final LifeCycleState oldState = container.getState();
containerStateManager.updateContainerState(containerID, event);
final LifeCycleState newState = container.getState();
if (!skipPipelineToContainerRemove) {
if (oldState == LifeCycleState.OPEN &&
newState != LifeCycleState.OPEN) {
pipelineManager
.removeContainerFromPipeline(container.getPipelineID(),
containerID);
}
}
containerStore.put(containerID, container);
return newState;
} catch (ContainerNotFoundException cnfe) {
throw new SCMException(
"Failed to update container state"
+ containerID
+ ", reason : container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
} finally {
lock.unlock();
}
}
/**
* Update deleteTransactionId according to deleteTransactionMap.
*
* @param deleteTransactionMap Maps the containerId to latest delete
* transaction id for the container.
* @throws IOException
*/
public void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
throws IOException {
if (deleteTransactionMap == null) {
return;
}
org.apache.hadoop.hdds.utils.db.BatchOperation batchOperation =
batchHandler.initBatchOperation();
lock.lock();
try {
for (Map.Entry<Long, Long> entry : deleteTransactionMap.entrySet()) {
long containerID = entry.getKey();
ContainerID containerIdObject = new ContainerID(containerID);
ContainerInfo containerInfo =
containerStore.get(containerIdObject);
if (containerInfo == null) {
throw new SCMException(
"Failed to increment number of deleted blocks for container "
+ containerID + ", reason : " + "container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
}
containerInfo.updateDeleteTransactionId(entry.getValue());
containerStore
.putWithBatch(batchOperation, containerIdObject, containerInfo);
}
batchHandler.commitBatchOperation(batchOperation);
containerStateManager
.updateDeleteTransactionId(deleteTransactionMap);
} finally {
lock.unlock();
}
}
/**
* Return a container matching the attributes specified.
*
* @param sizeRequired - Space needed in the Container.
* @param owner - Owner of the container - A specific nameservice.
* @param pipeline - Pipeline to which the container should belong.
* @return ContainerInfo, null if there is no match found.
*/
public ContainerInfo getMatchingContainer(final long sizeRequired,
String owner, Pipeline pipeline) {
return getMatchingContainer(sizeRequired, owner, pipeline,
Collections.emptySet());
}
@SuppressWarnings("squid:S2445")
public ContainerInfo getMatchingContainer(final long sizeRequired,
String owner, Pipeline pipeline,
Collection<ContainerID>
excludedContainers) {
NavigableSet<ContainerID> containerIDs;
ContainerInfo containerInfo;
try {
synchronized (pipeline) {
containerIDs = getContainersForOwner(pipeline, owner);
if (containerIDs.size() < numContainerPerVolume * pipelineManager.
getNumHealthyVolumes(pipeline)) {
containerInfo =
containerStateManager.allocateContainer(
pipelineManager, owner, pipeline);
// Add to DB
addContainerToDB(containerInfo);
} else {
containerIDs.removeAll(excludedContainers);
containerInfo =
containerStateManager.getMatchingContainer(
sizeRequired, owner, pipeline.getId(), containerIDs);
if (containerInfo == null) {
containerInfo =
containerStateManager.
allocateContainer(pipelineManager, owner,
pipeline);
// Add to DB
addContainerToDB(containerInfo);
}
}
containerStateManager.updateLastUsedMap(pipeline.getId(),
containerInfo.containerID(), owner);
// TODO: #CLUTIL cleanup entries in lastUsedMap
return containerInfo;
}
} catch (Exception e) {
LOG.warn("Container allocation failed for pipeline={} requiredSize={} {}",
pipeline, sizeRequired, e);
return null;
}
}
/**
* Add newly allocated container to container DB.
* @param containerInfo
* @throws IOException
*/
protected void addContainerToDB(ContainerInfo containerInfo)
throws IOException {
try {
containerStore
.put(new ContainerID(containerInfo.getContainerID()), containerInfo);
// Incrementing here, as allocateBlock to create a container calls
// getMatchingContainer() and finally calls this API to add newly
// created container to DB.
// Even allocateContainer calls this API to add newly allocated
// container to DB. So we need to increment metrics here.
scmContainerManagerMetrics.incNumSuccessfulCreateContainers();
} catch (IOException ex) {
// If adding to containerStore fails, we should remove the container
// from in-memory map.
scmContainerManagerMetrics.incNumFailureCreateContainers();
LOG.error("Add Container to DB failed for ContainerID #{}",
containerInfo.getContainerID());
try {
containerStateManager.removeContainer(containerInfo.containerID());
} catch (ContainerNotFoundException cnfe) {
// This should not happen, as we are removing after adding in to
// container state cmap.
}
throw ex;
}
}
/**
* Returns the container ID's matching with specified owner.
* @param pipeline
* @param owner
* @return NavigableSet<ContainerID>
*/
private NavigableSet<ContainerID> getContainersForOwner(
Pipeline pipeline, String owner) throws IOException {
NavigableSet<ContainerID> containerIDs =
pipelineManager.getContainersInPipeline(pipeline.getId());
Iterator<ContainerID> containerIDIterator = containerIDs.iterator();
while (containerIDIterator.hasNext()) {
ContainerID cid = containerIDIterator.next();
try {
if (!getContainer(cid).getOwner().equals(owner)) {
containerIDIterator.remove();
}
} catch (ContainerNotFoundException e) {
LOG.error("Could not find container info for container id={} {}", cid,
e);
containerIDIterator.remove();
}
}
return containerIDs;
}
/**
* Returns the latest list of DataNodes where replica for given containerId
* exist. Throws an SCMException if no entry is found for given containerId.
*
* @param containerID
* @return Set<DatanodeDetails>
*/
public Set<ContainerReplica> getContainerReplicas(
final ContainerID containerID) throws ContainerNotFoundException {
return containerStateManager.getContainerReplicas(containerID);
}
/**
* Add a container Replica for given DataNode.
*
* @param containerID
* @param replica
*/
public void updateContainerReplica(final ContainerID containerID,
final ContainerReplica replica) throws ContainerNotFoundException {
containerStateManager.updateContainerReplica(containerID, replica);
}
/**
* Remove a container Replica for given DataNode.
*
* @param containerID
* @param replica
* @return True of dataNode is removed successfully else false.
*/
public void removeContainerReplica(final ContainerID containerID,
final ContainerReplica replica)
throws ContainerNotFoundException, ContainerReplicaNotFoundException {
containerStateManager.removeContainerReplica(containerID, replica);
}
/**
* Closes this stream and releases any system resources associated with it.
* If the stream is
* already closed then invoking this method has no effect.
* <p>
* <p>As noted in {@link AutoCloseable#close()}, cases where the close may
* fail require careful
* attention. It is strongly advised to relinquish the underlying resources
* and to internally
* <em>mark</em> the {@code Closeable} as closed, prior to throwing the
* {@code IOException}.
*
* @throws IOException if an I/O error occurs
*/
@Override
public void close() throws IOException {
if (containerStateManager != null) {
containerStateManager.close();
}
if (scmContainerManagerMetrics != null) {
this.scmContainerManagerMetrics.unRegister();
}
}
public void notifyContainerReportProcessing(boolean isFullReport,
boolean success) {
if (isFullReport) {
if (success) {
scmContainerManagerMetrics.incNumContainerReportsProcessedSuccessful();
} else {
scmContainerManagerMetrics.incNumContainerReportsProcessedFailed();
}
} else {
if (success) {
scmContainerManagerMetrics.incNumICRReportsProcessedSuccessful();
} else {
scmContainerManagerMetrics.incNumICRReportsProcessedFailed();
}
}
}
protected PipelineManager getPipelineManager() {
return pipelineManager;
}
public Lock getLock() {
return lock;
}
}