Skip to content
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

HDDS-11209. Avoid insufficient EC pipelines in the container pipeline cache #6974

Merged
merged 6 commits into from
Aug 21, 2024
Merged
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 @@ -18,10 +18,12 @@

package org.apache.hadoop.ozone.om;

import com.google.common.collect.ImmutableMap;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.client.ContainerBlockID;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
Expand Down Expand Up @@ -89,8 +91,11 @@
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
Expand Down Expand Up @@ -145,10 +150,17 @@ public class TestOmContainerLocationCache {
private static ObjectStore objectStore;
private static XceiverClientGrpc mockDn1Protocol;
private static XceiverClientGrpc mockDn2Protocol;
private static XceiverClientGrpc mockDnEcProtocol;
private static final DatanodeDetails DN1 =
MockDatanodeDetails.createDatanodeDetails(UUID.randomUUID());
private static final DatanodeDetails DN2 =
MockDatanodeDetails.createDatanodeDetails(UUID.randomUUID());
private static final DatanodeDetails DN3 =
MockDatanodeDetails.createDatanodeDetails(UUID.randomUUID());
private static final DatanodeDetails DN4 =
MockDatanodeDetails.createDatanodeDetails(UUID.randomUUID());
private static final DatanodeDetails DN5 =
MockDatanodeDetails.createDatanodeDetails(UUID.randomUUID());
private static final AtomicLong CONTAINER_ID = new AtomicLong(1);


Expand Down Expand Up @@ -200,6 +212,8 @@ private static XceiverClientManager mockDataNodeClientFactory()
throws IOException {
mockDn1Protocol = spy(new XceiverClientGrpc(createPipeline(DN1), conf));
mockDn2Protocol = spy(new XceiverClientGrpc(createPipeline(DN2), conf));
mockDnEcProtocol = spy(new XceiverClientGrpc(createEcPipeline(
ImmutableMap.of(DN1, 1, DN2, 2, DN3, 3, DN4, 4, DN5, 5)), conf));
XceiverClientManager manager = mock(XceiverClientManager.class);
when(manager.acquireClient(argThat(matchEmptyPipeline())))
.thenCallRealMethod();
Expand All @@ -217,6 +231,11 @@ private static XceiverClientManager mockDataNodeClientFactory()
.thenReturn(mockDn2Protocol);
when(manager.acquireClientForReadData(argThat(matchPipeline(DN2))))
.thenReturn(mockDn2Protocol);

when(manager.acquireClient(argThat(matchEcPipeline())))
.thenReturn(mockDnEcProtocol);
when(manager.acquireClientForReadData(argThat(matchEcPipeline())))
.thenReturn(mockDnEcProtocol);
return manager;
}

Expand All @@ -231,6 +250,11 @@ private static ArgumentMatcher<Pipeline> matchPipeline(DatanodeDetails dn) {
&& argument.getNodes().get(0).getUuid().equals(dn.getUuid());
}

private static ArgumentMatcher<Pipeline> matchEcPipeline() {
return argument -> argument != null && !argument.getNodes().isEmpty()
&& argument.getReplicationConfig() instanceof ECReplicationConfig;
}

private static void createBucket(String volumeName, String bucketName,
boolean isVersionEnabled)
throws IOException {
Expand All @@ -256,12 +280,14 @@ private static void createVolume(String volumeName) throws IOException {
public void beforeEach() throws IOException {
CONTAINER_ID.getAndIncrement();
reset(mockScmBlockLocationProtocol, mockScmContainerClient,
mockDn1Protocol, mockDn2Protocol);
mockDn1Protocol, mockDn2Protocol, mockDnEcProtocol);
InnerNode.Factory factory = InnerNodeImpl.FACTORY;
when(mockScmBlockLocationProtocol.getNetworkTopology()).thenReturn(
factory.newInnerNode("", "", null, NetConstants.ROOT_LEVEL, 1));
when(mockDn1Protocol.getPipeline()).thenReturn(createPipeline(DN1));
when(mockDn2Protocol.getPipeline()).thenReturn(createPipeline(DN2));
when(mockDnEcProtocol.getPipeline()).thenReturn(createEcPipeline(
ImmutableMap.of(DN1, 1, DN2, 2, DN3, 3, DN4, 4, DN5, 5)));
}

/**
Expand Down Expand Up @@ -575,6 +601,48 @@ public void containerRefreshedOnEmptyPipelines() throws Exception {
.getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get()));
}

@Test
public void containerRefreshedOnInsufficientEcPipelines() throws Exception {
int chunkSize = 1024 * 1024;
int dataBlocks = 3;
int parityBlocks = 2;
int inputSize = chunkSize * dataBlocks;
byte[][] inputChunks = new byte[dataBlocks][chunkSize];

mockScmAllocationEcPipeline(CONTAINER_ID.get(), 1L);
mockWriteChunkResponse(mockDnEcProtocol);
mockPutBlockResponse(mockDnEcProtocol, CONTAINER_ID.get(), 1L, null);

OzoneBucket bucket = objectStore.getVolume(VOLUME_NAME).getBucket(BUCKET_NAME);

String keyName = "ecKey";
try (OzoneOutputStream os = bucket.createKey(keyName, inputSize,
new ECReplicationConfig(dataBlocks, parityBlocks, ECReplicationConfig.EcCodec.RS,
chunkSize), new HashMap<>())) {
for (int i = 0; i < dataBlocks; i++) {
os.write(inputChunks[i]);
}
}

// case1: pipeline replicaIndexes missing some data indexes, should not cache
mockScmGetContainerEcPipeline(CONTAINER_ID.get(), ImmutableMap.of(DN1, 1, DN2, 2, DN4, 4));
bucket.getKey(keyName);
verify(mockScmContainerClient, times(1))
.getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get()));
bucket.getKey(keyName);
verify(mockScmContainerClient, times(2))
.getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get()));

// case2: pipeline replicaIndexes contain all data indexes, should cache
mockScmGetContainerEcPipeline(CONTAINER_ID.get(), ImmutableMap.of(DN1, 1, DN2, 2, DN3, 3, DN4, 4));
bucket.getKey(keyName);
verify(mockScmContainerClient, times(3))
.getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get()));
bucket.getKey(keyName);
verify(mockScmContainerClient, times(3))
.getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get()));
}

private void mockPutBlockResponse(XceiverClientSpi mockDnProtocol,
long containerId, long localId,
byte[] data)
Expand Down Expand Up @@ -639,6 +707,22 @@ private void mockScmAllocationOnDn1(long containerID,
.thenReturn(Collections.singletonList(block));
}

private void mockScmAllocationEcPipeline(long containerID, long localId)
throws IOException {
ContainerBlockID blockId = new ContainerBlockID(containerID, localId);
AllocatedBlock block = new AllocatedBlock.Builder()
.setPipeline(createEcPipeline(ImmutableMap.of(DN1, 1, DN2, 2, DN3, 3, DN4, 4, DN5, 5)))
.setContainerBlockID(blockId)
.build();
when(mockScmBlockLocationProtocol
.allocateBlock(anyLong(), anyInt(),
any(ECReplicationConfig.class),
anyString(),
any(ExcludeList.class),
anyString()))
.thenReturn(Collections.singletonList(block));
}

private void mockScmGetContainerPipeline(long containerId,
DatanodeDetails dn)
throws IOException {
Expand Down Expand Up @@ -668,6 +752,20 @@ private void mockScmGetContainerPipelineEmpty(long containerId)
newHashSet(containerId))).thenReturn(containerWithPipelines);
}

private void mockScmGetContainerEcPipeline(long containerId, Map<DatanodeDetails, Integer> indexes)
throws IOException {
Pipeline pipeline = createEcPipeline(indexes);
ContainerInfo containerInfo = new ContainerInfo.Builder()
.setContainerID(containerId)
.setPipelineID(pipeline.getId()).build();
List<ContainerWithPipeline> containerWithPipelines =
Collections.singletonList(
new ContainerWithPipeline(containerInfo, pipeline));

when(mockScmContainerClient.getContainerWithPipelineBatch(
newHashSet(containerId))).thenReturn(containerWithPipelines);
}

private void mockGetBlock(XceiverClientGrpc mockDnProtocol,
long containerId, long localId,
byte[] data,
Expand Down Expand Up @@ -766,4 +864,14 @@ private static Pipeline createPipeline(List<DatanodeDetails> nodes) {
.setNodes(nodes)
.build();
}

private static Pipeline createEcPipeline(Map<DatanodeDetails, Integer> indexes) {
return Pipeline.newBuilder()
.setState(Pipeline.PipelineState.OPEN)
.setId(PipelineID.randomId())
.setReplicationConfig(new ECReplicationConfig(3, 2))
.setReplicaIndexes(indexes)
.setNodes(new ArrayList<>(indexes.keySet()))
.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@
import com.google.common.cache.CacheLoader;
import com.google.common.cache.CacheLoader.InvalidCacheLoadException;
import com.google.common.cache.LoadingCache;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
Expand Down Expand Up @@ -113,12 +115,29 @@ public Map<Long, Pipeline> getContainerLocations(Iterable<Long> containerIds,
}
try {
Map<Long, Pipeline> result = containerLocationCache.getAll(containerIds);
// Don't keep empty pipelines in the cache.
List<Long> emptyPipelines = result.entrySet().stream()
.filter(e -> e.getValue().isEmpty())
// Don't keep empty pipelines or insufficient EC pipelines in the cache.
List<Long> uncachePipelines = result.entrySet().stream()
.filter(e -> {
Pipeline pipeline = e.getValue();
// filter empty pipelines
if (pipeline.isEmpty()) {
return true;
}
// filter insufficient EC pipelines which missing any data index
ReplicationConfig repConfig = pipeline.getReplicationConfig();
if (repConfig instanceof ECReplicationConfig) {
int d = ((ECReplicationConfig) repConfig).getData();
for (int i = 1; i <= d; i++) {
if (!pipeline.getReplicaIndexes().containsValue(i)) {
return true;
}
}
}
return false;
})
.map(Map.Entry::getKey)
.collect(Collectors.toList());
containerLocationCache.invalidateAll(emptyPipelines);
containerLocationCache.invalidateAll(uncachePipelines);
return result;
} catch (ExecutionException e) {
return handleCacheExecutionException(e);
Expand Down