Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -614,6 +614,7 @@ public final class ScmConfigKeys {
"ozone.scm.ha.dbtransactionbuffer.flush.interval";
public static final long
OZONE_SCM_HA_DBTRANSACTIONBUFFER_FLUSH_INTERVAL_DEFAULT = 600 * 1000L;

/**
* Never constructed.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -340,7 +340,10 @@ public ContainerInfo getMatchingContainer(final long size, final String owner,
}
} catch (Exception e) {
LOG.warn("Container allocation failed on pipeline={}", pipeline, e);
return null;
return new ContainerInfo.Builder()
.setContainerID(0)
.setPipelineID(pipeline.getId())
.build();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -207,6 +207,9 @@ private ContainerInfo allocateContainer(ReplicationConfig repConfig,
excludedNodes, Collections.emptyList());
ContainerInfo container =
containerManager.getMatchingContainer(size, owner, newPipeline);
if (container != null && container.getContainerID() == -1) {
container = null;
}
pipelineManager.openPipeline(newPipeline.getId());
LOG.info("Created and opened new pipeline {}", newPipeline);
return container;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,9 @@ public class WritableRatisContainerProvider
private final PipelineManager pipelineManager;
private final PipelineChoosePolicy pipelineChoosePolicy;
private final ContainerManager containerManager;
// added in HDDS-8982, it shouldn't be exposed to users to configure
private static int maxRetryGetContainer = 4096;


public WritableRatisContainerProvider(ConfigurationSource conf,
PipelineManager pipelineManager,
Expand Down Expand Up @@ -85,8 +88,8 @@ public ContainerInfo getContainer(final long size,
//TODO we need to continue the refactor to use repConfig everywhere
//in downstream managers.


while (true) {
int currentCount = 0;
while (currentCount < getMaxRetryGetContainer()) {
List<Pipeline> availablePipelines;
Pipeline pipeline;
// Acquire pipeline manager lock, to avoid any updates to pipeline
Expand All @@ -103,7 +106,13 @@ public ContainerInfo getContainer(final long size,
excludeList);
}
if (containerInfo != null) {
return containerInfo;
// selectContainer returns containerID with 0 whenever
// container allocation failed on selected pipeline
if (containerInfo.getContainerID() != 0) {
return containerInfo;
} else {
excludeList.addPipeline(containerInfo.getPipelineID());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Trying to exclude the faulty pipeline may not work, because:

if (pipelines.size() == 0 && !excludeList.isEmpty()) {
// if no pipelines can be found, try finding pipeline without
// exclusion
pipelines = pipelineManager.getPipelines(repConfig, pipelineState);
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah, the reason I add it in the first part of findPipelinesByState(Line 100) of this method is to prevent that broken pipeline from being selected in the second part of findPipelinesByState (Line 164 , after pipelineManager.createPipeline) I just feel it's nice to have that, but it could be removed too. I'm open to any suggestion whether to keep it or not here. Thanks!

}
}
} finally {
pipelineManager.releaseReadLock();
Expand Down Expand Up @@ -164,12 +173,17 @@ public ContainerInfo getContainer(final long size,
containerInfo = selectContainer(availablePipelines, size, owner,
excludeList);
if (containerInfo != null) {
return containerInfo;
if (containerInfo.getContainerID() != 0) {
return containerInfo;
} else {
excludeList.addPipeline(containerInfo.getPipelineID());
}
}
} finally {
pipelineManager.releaseReadLock();
}
}
currentCount++;
}

// we have tried all strategies we know but somehow we are not able
Expand Down Expand Up @@ -197,7 +211,7 @@ private List<Pipeline> findPipelinesByState(
return pipelines;
}

private ContainerInfo selectContainer(List<Pipeline> availablePipelines,
protected ContainerInfo selectContainer(List<Pipeline> availablePipelines,
long size, String owner, ExcludeList excludeList) {
Pipeline pipeline;
ContainerInfo containerInfo;
Expand All @@ -216,4 +230,8 @@ private ContainerInfo selectContainer(List<Pipeline> availablePipelines,

}

public int getMaxRetryGetContainer() {
return maxRetryGetContainer;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,181 @@
/*
* 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.pipeline;

import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.PipelineChoosePolicy;
import org.apache.hadoop.hdds.scm.PipelineRequestInformation;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
import org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.HealthyPipelineChoosePolicy;
import org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.RandomPipelineChoosePolicy;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.MethodSource;
import org.junit.runners.Parameterized;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.UUID;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyCollection;
import static org.mockito.ArgumentMatchers.anyList;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anySet;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;


/**
* Tests to validate the WritableRatisContainerProvider works correctly.
*/
public class TestWritableRatisContainerProvider {

private static final String OWNER = "SCM";
private PipelineManager pipelineManager;
private ContainerManager containerManager;
private OzoneConfiguration conf;
private WritableRatisContainerProvider provider;
private ReplicationConfig repConfig;

private int testSCMGetContainerMaxRetry;

@Parameterized.Parameters
public static Collection<PipelineChoosePolicy> policies() {
Collection<PipelineChoosePolicy> policies = new ArrayList<>();

Pipeline pipeline = mock(Pipeline.class);
RandomPipelineChoosePolicy randomPipelineChoosePolicy =
mock(RandomPipelineChoosePolicy.class);
when(randomPipelineChoosePolicy.choosePipeline(anyList(),
any(PipelineRequestInformation.class))).thenReturn(pipeline);

HealthyPipelineChoosePolicy healthyPipelineChoosePolicy =
mock(HealthyPipelineChoosePolicy.class);
when(healthyPipelineChoosePolicy.choosePipeline(anyList(),
any(PipelineRequestInformation.class))).thenReturn(pipeline);

policies.add(randomPipelineChoosePolicy);
policies.add(healthyPipelineChoosePolicy);

return policies;
}

void setup(PipelineChoosePolicy policy) {
conf = new OzoneConfiguration();
testSCMGetContainerMaxRetry = 3;

repConfig = mock(ReplicationConfig.class);
pipelineManager = mock(PipelineManager.class);
containerManager = mock(ContainerManager.class);
provider = spy(new WritableRatisContainerProvider(conf,
pipelineManager, containerManager, policy));
when(provider.getMaxRetryGetContainer())
.thenReturn(testSCMGetContainerMaxRetry);
}

@ParameterizedTest
@MethodSource("policies")
public void testSelectContainerShouldNoMoreThanMaxRetry(
PipelineChoosePolicy policy) {
setup(policy);
List<Pipeline> pipelines = mock(ArrayList.class);
when(pipelines.size()).thenReturn(3);
when(pipelineManager.getPipelines(any(ReplicationConfig.class),
any(Pipeline.PipelineState.class), anyCollection(),
anyCollection())).thenReturn(pipelines);
ContainerInfo containerInfo = mock(ContainerInfo.class);
when(containerInfo.getContainerID()).thenReturn(0L);
when(containerManager.getMatchingContainer(anyLong(), anyString(),
any(Pipeline.class), anySet())).thenReturn(containerInfo);

ExcludeList exclude = mock(ExcludeList.class);

try {
provider.getContainer(
1, repConfig, OWNER, exclude);
fail("expected IOException");
} catch (IOException e) {
assertTrue(e.getMessage()
.contains("Unable to allocate a container to the block"));
}

verify(provider, times(testSCMGetContainerMaxRetry))
.selectContainer(anyList(), anyLong(),
anyString(), any(ExcludeList.class));
}


@ParameterizedTest
@MethodSource("policies")
public void testSelectContainerShouldNoMoreThanMaxRetryAfterCreateNewPipeline(
PipelineChoosePolicy policy) throws IOException {
setup(policy);
List<Pipeline> emptyPipelines = new ArrayList<>();
List<Pipeline> pipelines = mock(ArrayList.class);
when(pipelines.size()).thenReturn(3);
when(pipelineManager.getPipelines(any(ReplicationConfig.class),
any(Pipeline.PipelineState.class), anyCollection(), anyCollection()))
.thenReturn(emptyPipelines, pipelines);

Pipeline pipeline1 = mock(Pipeline.class);
when(pipeline1.getId()).thenReturn(mock(PipelineID.class));
when(pipelineManager.createPipeline(any(ReplicationConfig.class)))
.thenReturn(pipeline1);

ContainerInfo containerInfo = mock(ContainerInfo.class);
when(containerInfo.getContainerID()).thenReturn(0L);

PipelineID pipelineID2 = PipelineID.valueOf(UUID.randomUUID());

when(containerInfo.getPipelineID()).thenReturn(pipelineID2);
when(containerManager.getMatchingContainer(anyLong(), anyString(), any(
Pipeline.class), anySet())).thenReturn(containerInfo);

ExcludeList exclude = new ExcludeList();

try {
provider.getContainer(
1, repConfig, OWNER, exclude);
fail("expected IOException");
} catch (IOException e) {
assertTrue(e.getMessage().
contains("Unable to allocate a container to the block"));
}

assertEquals(exclude.getPipelineIds(),
new HashSet<>(Arrays.asList(pipelineID2)));
verify(provider, times(testSCMGetContainerMaxRetry))
.selectContainer(anyList(), anyLong(), anyString(),
any(ExcludeList.class));
}

}