-
Notifications
You must be signed in to change notification settings - Fork 587
HDDS-12468. Check for space availability for all dns while container creation in pipeline #8663
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
beeca82
511a83f
b1d14ca
9af5a49
e58514f
24f1d42
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,6 +31,7 @@ | |
| import java.util.concurrent.locks.Lock; | ||
| import java.util.concurrent.locks.ReentrantLock; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.conf.StorageUnit; | ||
| import org.apache.hadoop.hdds.client.ECReplicationConfig; | ||
| import org.apache.hadoop.hdds.client.ReplicationConfig; | ||
| import org.apache.hadoop.hdds.protocol.proto.HddsProtos; | ||
|
|
@@ -80,6 +81,8 @@ public class ContainerManagerImpl implements ContainerManager { | |
| @SuppressWarnings("java:S2245") // no need for secure random | ||
| private final Random random = new Random(); | ||
|
|
||
| private final long maxContainerSize; | ||
|
|
||
| /** | ||
| * | ||
| */ | ||
|
|
@@ -109,6 +112,9 @@ public ContainerManagerImpl( | |
| .getInt(ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT, | ||
| ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT_DEFAULT); | ||
|
|
||
| maxContainerSize = (long) conf.getStorageSize(ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, | ||
| ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); | ||
|
|
||
| this.scmContainerManagerMetrics = SCMContainerManagerMetrics.create(); | ||
| } | ||
|
|
||
|
|
@@ -346,14 +352,24 @@ public ContainerInfo getMatchingContainer(final long size, final String owner, | |
| synchronized (pipeline.getId()) { | ||
| containerIDs = getContainersForOwner(pipeline, owner); | ||
| if (containerIDs.size() < getOpenContainerCountPerPipeline(pipeline)) { | ||
| allocateContainer(pipeline, owner); | ||
| containerIDs = getContainersForOwner(pipeline, owner); | ||
| if (pipelineManager.hasEnoughSpace(pipeline, maxContainerSize)) { | ||
| allocateContainer(pipeline, owner); | ||
| containerIDs = getContainersForOwner(pipeline, owner); | ||
| } else { | ||
| LOG.debug("Cannot allocate a new container because pipeline {} does not have the required space {}.", | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please add the if (LOG.isDebugEnabled) check.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. AFAIK the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yea if the log parameters are just simple references to be passed, the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good to know that slf4j with "{}" will delay the string construction, thanks! |
||
| pipeline, maxContainerSize); | ||
| } | ||
| } | ||
| containerIDs.removeAll(excludedContainerIDs); | ||
| containerInfo = containerStateManager.getMatchingContainer( | ||
| size, owner, pipeline.getId(), containerIDs); | ||
| if (containerInfo == null) { | ||
| containerInfo = allocateContainer(pipeline, owner); | ||
| if (pipelineManager.hasEnoughSpace(pipeline, maxContainerSize)) { | ||
| containerInfo = allocateContainer(pipeline, owner); | ||
| } else { | ||
| LOG.debug("Cannot allocate a new container because pipeline {} does not have the required space {}.", | ||
| pipeline, maxContainerSize); | ||
| } | ||
| } | ||
| return containerInfo; | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -203,8 +203,14 @@ private ContainerInfo allocateContainer(ReplicationConfig repConfig, | |
|
|
||
| Pipeline newPipeline = pipelineManager.createPipeline(repConfig, | ||
| excludedNodes, Collections.emptyList()); | ||
| // the returned ContainerInfo should not be null (due to not enough space in the Datanodes specifically) because | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could newPipeline be null when DN is close to full?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good point. I checked, pipeline will not be null. There's a |
||
| // this is a new pipeline and pipeline creation checks for sufficient space in the Datanodes | ||
| ContainerInfo container = | ||
| containerManager.getMatchingContainer(size, owner, newPipeline); | ||
| if (container == null) { | ||
| // defensive null handling | ||
| throw new IOException("Could not allocate a new container"); | ||
| } | ||
| pipelineManager.openPipeline(newPipeline.getId()); | ||
| LOG.info("Created and opened new pipeline {}", newPipeline); | ||
| return container; | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Use @nullable, which is very straightforward.