Skip to content

Commit be1ed89

Browse files
HDDS-15104. Refactor code related to container space management. (#10124)
1 parent 04d1044 commit be1ed89

24 files changed

Lines changed: 305 additions & 313 deletions

hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java

Lines changed: 2 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -31,15 +31,13 @@
3131
import java.util.concurrent.locks.Lock;
3232
import java.util.concurrent.locks.ReentrantLock;
3333
import org.apache.hadoop.conf.Configuration;
34-
import org.apache.hadoop.conf.StorageUnit;
3534
import org.apache.hadoop.hdds.client.ECReplicationConfig;
3635
import org.apache.hadoop.hdds.client.ReplicationConfig;
3736
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
3837
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerInfoProto;
3938
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
4039
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
4140
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
42-
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
4341
import org.apache.hadoop.hdds.scm.container.metrics.SCMContainerManagerMetrics;
4442
import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
4543
import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
@@ -79,8 +77,6 @@ public class ContainerManagerImpl implements ContainerManager {
7977
@SuppressWarnings("java:S2245") // no need for secure random
8078
private final Random random = new Random();
8179

82-
private final long maxContainerSize;
83-
8480
/**
8581
*
8682
*/
@@ -106,9 +102,6 @@ public ContainerManagerImpl(
106102
.setContainerReplicaPendingOps(containerReplicaPendingOps)
107103
.build();
108104

109-
maxContainerSize = (long) conf.getStorageSize(ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
110-
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
111-
112105
this.scmContainerManagerMetrics = SCMContainerManagerMetrics.create();
113106
}
114107

@@ -245,9 +238,8 @@ private ContainerInfo createContainer(Pipeline pipeline, String owner)
245238
private ContainerInfo allocateContainer(final Pipeline pipeline,
246239
final String owner)
247240
throws IOException {
248-
if (!pipelineManager.hasEnoughSpace(pipeline, maxContainerSize)) {
249-
LOG.debug("Cannot allocate a new container because pipeline {} does not have the required space {}.",
250-
pipeline, maxContainerSize);
241+
if (!pipelineManager.hasEnoughSpace(pipeline)) {
242+
LOG.debug("Cannot allocate a new container because pipeline {} does not have enough space.", pipeline);
251243
return null;
252244
}
253245

hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeInfo.java

Lines changed: 16 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.MetadataStorageReportProto;
3333
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
3434
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto;
35+
import org.apache.hadoop.hdds.scm.node.PendingContainerTracker.TwoWindowBucket;
3536
import org.apache.hadoop.util.Time;
3637
import org.slf4j.Logger;
3738
import org.slf4j.LoggerFactory;
@@ -50,6 +51,11 @@ public class DatanodeInfo extends DatanodeDetails {
5051
private long lastStatsUpdatedTime;
5152
private int failedVolumeCount;
5253

54+
/**
55+
* Two-window tumbling bucket for tracking pending container allocations on this datanode.
56+
*/
57+
private final TwoWindowBucket pendingContainerAllocations;
58+
5359
private List<StorageReportProto> storageReports;
5460
private List<MetadataStorageReportProto> metadataStorageReports;
5561
private LayoutVersionProto lastKnownLayoutVersion;
@@ -64,7 +70,7 @@ public class DatanodeInfo extends DatanodeDetails {
6470
* @param layoutInfo Details about the LayoutVersionProto
6571
*/
6672
public DatanodeInfo(DatanodeDetails datanodeDetails, NodeStatus nodeStatus,
67-
LayoutVersionProto layoutInfo) {
73+
LayoutVersionProto layoutInfo, long containerRollIntervalMs) {
6874
super(datanodeDetails);
6975
this.lock = new ReentrantReadWriteLock();
7076
this.lastHeartbeatTime = Time.monotonicNow();
@@ -75,6 +81,7 @@ public DatanodeInfo(DatanodeDetails datanodeDetails, NodeStatus nodeStatus,
7581
this.nodeStatus = nodeStatus;
7682
this.metadataStorageReports = Collections.emptyList();
7783
this.commandCounts = new HashMap<>();
84+
this.pendingContainerAllocations = new TwoWindowBucket(this.getID(), containerRollIntervalMs);
7885
}
7986

8087
/**
@@ -353,6 +360,14 @@ public int getCommandCount(SCMCommandProto.Type cmd) {
353360
}
354361
}
355362

363+
/**
364+
* Returns the {@link TwoWindowBucket} for this datanode.
365+
*/
366+
public TwoWindowBucket getPendingContainerAllocations() {
367+
pendingContainerAllocations.rollIfNeeded();
368+
return pendingContainerAllocations;
369+
}
370+
356371
@Override
357372
public int hashCode() {
358373
return super.hashCode();

hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -184,6 +184,19 @@ default int getAllNodeCount() {
184184
@Nullable
185185
DatanodeInfo getDatanodeInfo(DatanodeDetails dn);
186186

187+
/**
188+
* True if the node can accept another container of the given size.
189+
*/
190+
boolean hasSpaceForNewContainerAllocation(DatanodeID datanodeID);
191+
192+
/**
193+
* Records a pending container allocation for a single DataNode identified by its ID.
194+
*
195+
* @param datanodeID the ID of the DataNode receiving the allocation
196+
* @param containerID the container being allocated
197+
*/
198+
void recordPendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID);
199+
187200
/**
188201
* Return the node stat of the specified datanode.
189202
* @param datanodeDetails DatanodeDetails.

hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -124,6 +124,8 @@ public class NodeStateManager implements Runnable, Closeable {
124124
*/
125125
private final long deadNodeIntervalMs;
126126

127+
private final long containerRollIntervalMs = 5 * 60 * 1000; //TODO
128+
127129
/**
128130
* The future is used to pause/unpause the scheduled checks.
129131
*/
@@ -310,7 +312,7 @@ public void addNode(DatanodeDetails datanodeDetails,
310312

311313
private DatanodeInfo newDatanodeInfo(DatanodeDetails datanode, LayoutVersionProto layout) {
312314
final NodeStatus status = newNodeStatus(datanode, layout);
313-
return new DatanodeInfo(datanode, status, layout);
315+
return new DatanodeInfo(datanode, status, layout, containerRollIntervalMs);
314316
}
315317

316318
/**

hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java

Lines changed: 19 additions & 117 deletions
Original file line numberDiff line numberDiff line change
@@ -17,17 +17,13 @@
1717

1818
package org.apache.hadoop.hdds.scm.node;
1919

20-
import com.google.common.annotations.VisibleForTesting;
2120
import java.util.HashSet;
2221
import java.util.List;
2322
import java.util.Objects;
2423
import java.util.Set;
25-
import java.util.concurrent.ConcurrentHashMap;
26-
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
2724
import org.apache.hadoop.hdds.protocol.DatanodeID;
2825
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto;
2926
import org.apache.hadoop.hdds.scm.container.ContainerID;
30-
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
3127
import org.apache.hadoop.ozone.container.common.volume.VolumeUsage;
3228
import org.apache.hadoop.util.Time;
3329
import org.slf4j.Logger;
@@ -70,8 +66,6 @@ public class PendingContainerTracker {
7066

7167
private static final Logger LOG = LoggerFactory.getLogger(PendingContainerTracker.class);
7268

73-
private final DatanodeBuckets datanodeBuckets;
74-
7569
/**
7670
* Maximum container size in bytes.
7771
*/
@@ -86,13 +80,15 @@ public class PendingContainerTracker {
8680
* Two-window bucket for a single DataNode.
8781
* Contains current and previous window sets, plus last roll timestamp.
8882
*/
89-
private static class TwoWindowBucket {
83+
public static class TwoWindowBucket {
9084
private Set<ContainerID> currentWindow = new HashSet<>();
9185
private Set<ContainerID> previousWindow = new HashSet<>();
9286
private long lastRollTime = Time.monotonicNow();
9387
private final long rollIntervalMs;
88+
private final DatanodeID datanodeID;
9489

95-
TwoWindowBucket(long rollIntervalMs) {
90+
TwoWindowBucket(DatanodeID id, long rollIntervalMs) {
91+
this.datanodeID = id;
9692
this.rollIntervalMs = rollIntervalMs;
9793
}
9894

@@ -127,22 +123,22 @@ synchronized boolean contains(ContainerID containerID) {
127123
/**
128124
* Add container to current window.
129125
*/
130-
synchronized boolean add(ContainerID containerID, DatanodeID dnID) {
126+
synchronized boolean add(ContainerID containerID) {
131127
boolean added = currentWindow.add(containerID);
132128
LOG.debug("Recorded pending container {} on DataNode {}. Added={}, Total pending={}",
133-
containerID, dnID, added, getCount());
129+
containerID, datanodeID, added, getCount());
134130
return added;
135131
}
136132

137133
/**
138134
* Remove container from both windows.
139135
*/
140-
synchronized boolean remove(ContainerID containerID, DatanodeID dnID) {
136+
synchronized boolean remove(ContainerID containerID) {
141137
boolean removedFromCurrent = currentWindow.remove(containerID);
142138
boolean removedFromPrevious = previousWindow.remove(containerID);
143139
boolean removed = removedFromCurrent || removedFromPrevious;
144140
LOG.debug("Removed pending container {} from DataNode {}. Removed={}, Remaining={}",
145-
containerID, dnID, removed, getCount());
141+
containerID, datanodeID, removed, getCount());
146142
return removed;
147143
}
148144

@@ -154,63 +150,25 @@ synchronized int getCount() {
154150
}
155151
}
156152

157-
/**
158-
* Per-datanode two-window buckets.
159-
*/
160-
private static class DatanodeBuckets {
161-
private final ConcurrentHashMap<DatanodeID, TwoWindowBucket> map = new ConcurrentHashMap<>();
162-
private final long rollIntervalMs;
163-
164-
DatanodeBuckets(long rollIntervalMs) {
165-
this.rollIntervalMs = rollIntervalMs;
166-
}
167-
168-
TwoWindowBucket get(DatanodeID id) {
169-
final TwoWindowBucket bucket = map.compute(id, (k, b) -> b != null ? b : new TwoWindowBucket(rollIntervalMs));
170-
bucket.rollIfNeeded();
171-
return bucket;
172-
}
173-
174-
TwoWindowBucket get(DatanodeDetails dn) {
175-
Objects.requireNonNull(dn, "dn == null");
176-
return get(dn.getID());
177-
}
178-
}
179-
180-
public PendingContainerTracker(long maxContainerSize, long rollIntervalMs,
181-
SCMNodeMetrics metrics) {
182-
this.datanodeBuckets = new DatanodeBuckets(rollIntervalMs);
153+
public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, SCMNodeMetrics metrics) {
183154
this.maxContainerSize = maxContainerSize;
184155
this.metrics = metrics;
185156
LOG.info("PendingContainerTracker initialized with maxContainerSize={}B, rollInterval={}ms",
186157
maxContainerSize, rollIntervalMs);
187158
}
188159

189-
/**
190-
* Advances the two-window tumbling bucket for this datanode when the roll interval has elapsed.
191-
* Call on periodic paths (node report) so windows age even when there are no new
192-
* allocations or container reports touching this tracker.
193-
*/
194-
public void rollWindowsIfNeeded(DatanodeDetails node) {
195-
Objects.requireNonNull(node, "node == null");
196-
datanodeBuckets.get(node.getID());
197-
}
198-
199160
/**
200161
* Whether the datanode can fit another container of {@link #maxContainerSize} after accounting for
201162
* SCM pending allocations for {@code node} (this tracker) and usable space across volumes on
202-
* {@code datanodeInfo}. Pending bytes are {@link #getPendingContainerCount} × {@code maxContainerSize};
163+
* {@code datanodeInfo}. Pending bytes are count × {@code maxContainerSize};
203164
* effective allocatable space sums full-container slots per storage report.
204165
*
205-
* @param node identity used to look up pending allocations (same DN as {@code datanodeInfo})
206166
* @param datanodeInfo storage reports for the datanode
207167
*/
208-
public boolean hasEffectiveAllocatableSpaceForNewContainer(
209-
DatanodeDetails node, DatanodeInfo datanodeInfo) {
210-
Objects.requireNonNull(node, "node == null");
168+
public boolean hasEffectiveAllocatableSpaceForNewContainer(DatanodeInfo datanodeInfo) {
211169
Objects.requireNonNull(datanodeInfo, "datanodeInfo == null");
212170

213-
long pendingAllocationSize = getPendingContainerCount(node) * maxContainerSize;
171+
long pendingAllocationSize = datanodeInfo.getPendingContainerAllocations().getCount() * maxContainerSize;
214172
List<StorageReportProto> storageReports = datanodeInfo.getStorageReports();
215173
Objects.requireNonNull(storageReports, "storageReports == null");
216174
if (storageReports.isEmpty()) {
@@ -231,93 +189,37 @@ public boolean hasEffectiveAllocatableSpaceForNewContainer(
231189
return false;
232190
}
233191

234-
/**
235-
* Record a pending container allocation for all DataNodes in the pipeline.
236-
* Container is added to the current window.
237-
*
238-
* @param pipeline The pipeline where container is allocated
239-
* @param containerID The container being allocated
240-
*/
241-
public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) {
242-
Objects.requireNonNull(pipeline, "pipeline == null");
243-
Objects.requireNonNull(containerID, "containerID == null");
244-
245-
for (DatanodeDetails node : pipeline.getNodes()) {
246-
recordPendingAllocationForDatanode(node, containerID);
247-
}
248-
}
249-
250192
/**
251193
* Record a pending container allocation for a single DataNode.
252194
* Container is added to the current window.
253195
*
254-
* @param node The DataNode where container is being allocated/replicated
255196
* @param containerID The container being allocated/replicated
256197
*/
257-
public void recordPendingAllocationForDatanode(DatanodeDetails node, ContainerID containerID) {
258-
Objects.requireNonNull(node, "node == null");
198+
public void recordPendingAllocationForDatanode(DatanodeInfo datanodeInfo, ContainerID containerID) {
259199
Objects.requireNonNull(containerID, "containerID == null");
260-
261-
DatanodeID dnID = node.getID();
262-
boolean added = addContainerToBucket(containerID, dnID);
263-
200+
if (datanodeInfo == null) {
201+
return;
202+
}
203+
final boolean added = datanodeInfo.getPendingContainerAllocations().add(containerID);
264204
if (added && metrics != null) {
265205
metrics.incNumPendingContainersAdded();
266206
}
267207
}
268208

269-
private boolean addContainerToBucket(ContainerID containerID, DatanodeID dnID) {
270-
return datanodeBuckets.get(dnID).add(containerID, dnID);
271-
}
272-
273209
/**
274210
* Remove a pending container allocation from a specific DataNode.
275211
* Removes from both current and previous windows.
276212
* Called when container is confirmed.
277213
*
278-
* @param node The DataNode
279214
* @param containerID The container to remove from pending
280215
*/
281-
public void removePendingAllocation(DatanodeDetails node, ContainerID containerID) {
282-
Objects.requireNonNull(node, "node == null");
216+
public void removePendingAllocation(TwoWindowBucket bucket, ContainerID containerID) {
283217
Objects.requireNonNull(containerID, "containerID == null");
284218

285-
DatanodeID dnID = node.getID();
286-
boolean removed = removeContainerFromBucket(containerID, dnID);
219+
boolean removed = bucket.remove(containerID);
287220

288221
if (removed && metrics != null) {
289222
metrics.incNumPendingContainersRemoved();
290223
}
291224
}
292-
293-
private boolean removeContainerFromBucket(ContainerID containerID, DatanodeID dnID) {
294-
return datanodeBuckets.get(dnID).remove(containerID, dnID);
295-
}
296-
297-
/**
298-
* Number of pending container allocations for this datanode (union of current and previous
299-
* windows). This call may advance the internal tumbling window if the roll interval has elapsed.
300-
*
301-
* @param node The DataNode
302-
* @return Pending container count
303-
*/
304-
public long getPendingContainerCount(DatanodeDetails node) {
305-
Objects.requireNonNull(node, "node == null");
306-
return datanodeBuckets.get(node).getCount();
307-
}
308-
309-
/**
310-
* Whether container is in the current or previous window for this datanode.
311-
*/
312-
@VisibleForTesting
313-
public boolean containsPendingContainer(DatanodeDetails node, ContainerID containerID) {
314-
Objects.requireNonNull(node, "node == null");
315-
Objects.requireNonNull(containerID, "containerID == null");
316-
return datanodeBuckets.get(node).contains(containerID);
317-
}
318-
319-
@VisibleForTesting
320-
public SCMNodeMetrics getMetrics() {
321-
return metrics;
322-
}
323225
}

0 commit comments

Comments
 (0)