HDDS-1690. ContainerController should provide a way to retrieve containers per volume (#986)
This commit is contained in:
parent
cba13c7f83
commit
8194a1196e
@ -29,6 +29,7 @@
|
|||||||
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
||||||
import org.apache.hadoop.ozone.container.common
|
import org.apache.hadoop.ozone.container.common
|
||||||
.interfaces.ContainerDeletionChoosingPolicy;
|
.interfaces.ContainerDeletionChoosingPolicy;
|
||||||
|
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
@ -128,6 +129,24 @@ public Iterator<Container> getContainerIterator() {
|
|||||||
return containerMap.values().iterator();
|
return containerMap.values().iterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return an iterator of containers associated with the specified volume.
|
||||||
|
*
|
||||||
|
* @param volume the HDDS volume which should be used to filter containers
|
||||||
|
* @return {@literal Iterator<Container>}
|
||||||
|
*/
|
||||||
|
public Iterator<Container> getContainerIterator(HddsVolume volume) {
|
||||||
|
Preconditions.checkNotNull(volume);
|
||||||
|
Preconditions.checkNotNull(volume.getStorageID());
|
||||||
|
String volumeUuid = volume.getStorageID();
|
||||||
|
return containerMap.values()
|
||||||
|
.stream()
|
||||||
|
.filter(x -> volumeUuid.equals(
|
||||||
|
x.getContainerData().getVolume()
|
||||||
|
.getStorageID()))
|
||||||
|
.iterator();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return an containerMap iterator over {@link ContainerSet#containerMap}.
|
* Return an containerMap iterator over {@link ContainerSet#containerMap}.
|
||||||
* @return containerMap Iterator
|
* @return containerMap Iterator
|
||||||
|
@ -26,6 +26,7 @@
|
|||||||
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
|
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
|
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
|
||||||
|
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
|
||||||
import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
|
import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
|
||||||
|
|
||||||
import java.io.FileInputStream;
|
import java.io.FileInputStream;
|
||||||
@ -140,4 +141,15 @@ private Handler getHandler(final Container container) {
|
|||||||
public Iterator<Container> getContainers() {
|
public Iterator<Container> getContainers() {
|
||||||
return containerSet.getContainerIterator();
|
return containerSet.getContainerIterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return an iterator of containers which are associated with the specified
|
||||||
|
* <code>volume</code>.
|
||||||
|
*
|
||||||
|
* @param volume the HDDS volume which should be used to filter containers
|
||||||
|
* @return {@literal Iterator<Container>}
|
||||||
|
*/
|
||||||
|
public Iterator<Container> getContainers(HddsVolume volume) {
|
||||||
|
return containerSet.getContainerIterator(volume);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -25,10 +25,12 @@
|
|||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
||||||
|
|
||||||
|
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
|
||||||
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
|
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
|
||||||
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
|
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
@ -135,6 +137,48 @@ public void testIteratorsAndCount() throws StorageContainerException {
|
|||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIteratorPerVolume() throws StorageContainerException {
|
||||||
|
HddsVolume vol1 = Mockito.mock(HddsVolume.class);
|
||||||
|
Mockito.when(vol1.getStorageID()).thenReturn("uuid-1");
|
||||||
|
HddsVolume vol2 = Mockito.mock(HddsVolume.class);
|
||||||
|
Mockito.when(vol2.getStorageID()).thenReturn("uuid-2");
|
||||||
|
|
||||||
|
ContainerSet containerSet = new ContainerSet();
|
||||||
|
for (int i=0; i<10; i++) {
|
||||||
|
KeyValueContainerData kvData = new KeyValueContainerData(i,
|
||||||
|
(long) StorageUnit.GB.toBytes(5), UUID.randomUUID().toString(),
|
||||||
|
UUID.randomUUID().toString());
|
||||||
|
if (i%2 == 0) {
|
||||||
|
kvData.setVolume(vol1);
|
||||||
|
} else {
|
||||||
|
kvData.setVolume(vol2);
|
||||||
|
}
|
||||||
|
kvData.setState(ContainerProtos.ContainerDataProto.State.CLOSED);
|
||||||
|
KeyValueContainer kv = new KeyValueContainer(kvData, new
|
||||||
|
OzoneConfiguration());
|
||||||
|
containerSet.addContainer(kv);
|
||||||
|
}
|
||||||
|
|
||||||
|
Iterator<Container> iter1 = containerSet.getContainerIterator(vol1);
|
||||||
|
int count1 = 0;
|
||||||
|
while (iter1.hasNext()) {
|
||||||
|
Container c = iter1.next();
|
||||||
|
assertTrue((c.getContainerData().getContainerID() % 2) == 0);
|
||||||
|
count1++;
|
||||||
|
}
|
||||||
|
assertEquals(5, count1);
|
||||||
|
|
||||||
|
Iterator<Container> iter2 = containerSet.getContainerIterator(vol2);
|
||||||
|
int count2 = 0;
|
||||||
|
while (iter2.hasNext()) {
|
||||||
|
Container c = iter2.next();
|
||||||
|
assertTrue((c.getContainerData().getContainerID() % 2) == 1);
|
||||||
|
count2++;
|
||||||
|
}
|
||||||
|
assertEquals(5, count2);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGetContainerReport() throws IOException {
|
public void testGetContainerReport() throws IOException {
|
||||||
|
Loading…
Reference in New Issue
Block a user