|
@@ -18,6 +18,8 @@
|
|
|
|
|
|
package org.apache.hadoop.ozone.recon.spi.impl;
|
|
package org.apache.hadoop.ozone.recon.spi.impl;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DB_DIR;
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
import java.io.File;
|
|
import java.io.File;
|
|
@@ -28,10 +30,9 @@ import java.util.Map;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
|
|
import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
|
|
import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
|
|
import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
|
|
-import org.apache.hadoop.utils.MetaStoreIterator;
|
|
|
|
-import org.apache.hadoop.utils.MetadataStore;
|
|
|
|
-import org.apache.hadoop.utils.MetadataStoreBuilder;
|
|
|
|
|
|
+import org.apache.hadoop.utils.db.DBStore;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
|
|
+import org.junit.Assert;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Rule;
|
|
import org.junit.Rule;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
@@ -40,6 +41,7 @@ import org.junit.rules.TemporaryFolder;
|
|
import com.google.inject.AbstractModule;
|
|
import com.google.inject.AbstractModule;
|
|
import com.google.inject.Guice;
|
|
import com.google.inject.Guice;
|
|
import com.google.inject.Injector;
|
|
import com.google.inject.Injector;
|
|
|
|
+import com.google.inject.Singleton;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Unit Tests for ContainerDBServiceProviderImpl.
|
|
* Unit Tests for ContainerDBServiceProviderImpl.
|
|
@@ -49,28 +51,27 @@ public class TestContainerDBServiceProviderImpl {
|
|
@Rule
|
|
@Rule
|
|
public TemporaryFolder tempFolder = new TemporaryFolder();
|
|
public TemporaryFolder tempFolder = new TemporaryFolder();
|
|
|
|
|
|
- private MetadataStore containerDBStore;
|
|
|
|
- private ContainerDBServiceProvider containerDbServiceProvider
|
|
|
|
- = new ContainerDBServiceProviderImpl();
|
|
|
|
|
|
+ private ContainerDBServiceProvider containerDbServiceProvider;
|
|
private Injector injector;
|
|
private Injector injector;
|
|
|
|
|
|
@Before
|
|
@Before
|
|
public void setUp() throws IOException {
|
|
public void setUp() throws IOException {
|
|
tempFolder.create();
|
|
tempFolder.create();
|
|
- File dbDir = tempFolder.getRoot();
|
|
|
|
- containerDBStore = MetadataStoreBuilder.newBuilder()
|
|
|
|
- .setConf(new OzoneConfiguration())
|
|
|
|
- .setCreateIfMissing(true)
|
|
|
|
- .setDbFile(dbDir)
|
|
|
|
- .build();
|
|
|
|
injector = Guice.createInjector(new AbstractModule() {
|
|
injector = Guice.createInjector(new AbstractModule() {
|
|
@Override
|
|
@Override
|
|
protected void configure() {
|
|
protected void configure() {
|
|
- bind(MetadataStore.class).toInstance(containerDBStore);
|
|
|
|
- bind(ContainerDBServiceProvider.class)
|
|
|
|
- .toInstance(containerDbServiceProvider);
|
|
|
|
|
|
+ File dbDir = tempFolder.getRoot();
|
|
|
|
+ OzoneConfiguration configuration = new OzoneConfiguration();
|
|
|
|
+ configuration.set(OZONE_RECON_DB_DIR, dbDir.getAbsolutePath());
|
|
|
|
+ bind(OzoneConfiguration.class).toInstance(configuration);
|
|
|
|
+ bind(DBStore.class).toProvider(ReconContainerDBProvider.class).in(
|
|
|
|
+ Singleton.class);
|
|
|
|
+ bind(ContainerDBServiceProvider.class).to(
|
|
|
|
+ ContainerDBServiceProviderImpl.class).in(Singleton.class);
|
|
}
|
|
}
|
|
});
|
|
});
|
|
|
|
+ containerDbServiceProvider = injector.getInstance(
|
|
|
|
+ ContainerDBServiceProvider.class);
|
|
}
|
|
}
|
|
|
|
|
|
@After
|
|
@After
|
|
@@ -78,6 +79,55 @@ public class TestContainerDBServiceProviderImpl {
|
|
tempFolder.delete();
|
|
tempFolder.delete();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testInitNewContainerDB() throws Exception {
|
|
|
|
+ long containerId = System.currentTimeMillis();
|
|
|
|
+ Map<ContainerKeyPrefix, Integer> prefixCounts = new HashMap<>();
|
|
|
|
+
|
|
|
|
+ ContainerKeyPrefix ckp1 = new ContainerKeyPrefix(containerId,
|
|
|
|
+ "V1/B1/K1", 0);
|
|
|
|
+ prefixCounts.put(ckp1, 1);
|
|
|
|
+
|
|
|
|
+ ContainerKeyPrefix ckp2 = new ContainerKeyPrefix(containerId,
|
|
|
|
+ "V1/B1/K2", 0);
|
|
|
|
+ prefixCounts.put(ckp2, 2);
|
|
|
|
+
|
|
|
|
+ ContainerKeyPrefix ckp3 = new ContainerKeyPrefix(containerId,
|
|
|
|
+ "V1/B2/K3", 0);
|
|
|
|
+ prefixCounts.put(ckp3, 3);
|
|
|
|
+
|
|
|
|
+ for (ContainerKeyPrefix prefix : prefixCounts.keySet()) {
|
|
|
|
+ containerDbServiceProvider.storeContainerKeyMapping(
|
|
|
|
+ prefix, prefixCounts.get(prefix));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ assertEquals(1, containerDbServiceProvider
|
|
|
|
+ .getCountForForContainerKeyPrefix(ckp1).intValue());
|
|
|
|
+
|
|
|
|
+ prefixCounts.clear();
|
|
|
|
+ prefixCounts.put(ckp2, 12);
|
|
|
|
+ prefixCounts.put(ckp3, 13);
|
|
|
|
+ ContainerKeyPrefix ckp4 = new ContainerKeyPrefix(containerId,
|
|
|
|
+ "V1/B3/K1", 0);
|
|
|
|
+ prefixCounts.put(ckp4, 14);
|
|
|
|
+ ContainerKeyPrefix ckp5 = new ContainerKeyPrefix(containerId,
|
|
|
|
+ "V1/B3/K2", 0);
|
|
|
|
+ prefixCounts.put(ckp5, 15);
|
|
|
|
+
|
|
|
|
+ containerDbServiceProvider.initNewContainerDB(prefixCounts);
|
|
|
|
+ Map<ContainerKeyPrefix, Integer> keyPrefixesForContainer =
|
|
|
|
+ containerDbServiceProvider.getKeyPrefixesForContainer(containerId);
|
|
|
|
+
|
|
|
|
+ assertEquals(4, keyPrefixesForContainer.size());
|
|
|
|
+ assertEquals(12, keyPrefixesForContainer.get(ckp2).intValue());
|
|
|
|
+ assertEquals(13, keyPrefixesForContainer.get(ckp3).intValue());
|
|
|
|
+ assertEquals(14, keyPrefixesForContainer.get(ckp4).intValue());
|
|
|
|
+ assertEquals(15, keyPrefixesForContainer.get(ckp5).intValue());
|
|
|
|
+
|
|
|
|
+ assertEquals(0, containerDbServiceProvider
|
|
|
|
+ .getCountForForContainerKeyPrefix(ckp1).intValue());
|
|
|
|
+ }
|
|
|
|
+
|
|
@Test
|
|
@Test
|
|
public void testStoreContainerKeyMapping() throws Exception {
|
|
public void testStoreContainerKeyMapping() throws Exception {
|
|
|
|
|
|
@@ -89,19 +139,23 @@ public class TestContainerDBServiceProviderImpl {
|
|
|
|
|
|
for (String prefix : prefixCounts.keySet()) {
|
|
for (String prefix : prefixCounts.keySet()) {
|
|
ContainerKeyPrefix containerKeyPrefix = new ContainerKeyPrefix(
|
|
ContainerKeyPrefix containerKeyPrefix = new ContainerKeyPrefix(
|
|
- containerId, prefix);
|
|
|
|
|
|
+ containerId, prefix, 0);
|
|
containerDbServiceProvider.storeContainerKeyMapping(
|
|
containerDbServiceProvider.storeContainerKeyMapping(
|
|
containerKeyPrefix, prefixCounts.get(prefix));
|
|
containerKeyPrefix, prefixCounts.get(prefix));
|
|
}
|
|
}
|
|
|
|
|
|
- int count = 0;
|
|
|
|
- MetaStoreIterator<MetadataStore.KeyValue> iterator =
|
|
|
|
- containerDBStore.iterator();
|
|
|
|
- while (iterator.hasNext()) {
|
|
|
|
- iterator.next();
|
|
|
|
- count++;
|
|
|
|
- }
|
|
|
|
- assertTrue(count == 3);
|
|
|
|
|
|
+ Assert.assertTrue(
|
|
|
|
+ containerDbServiceProvider.getCountForForContainerKeyPrefix(
|
|
|
|
+ new ContainerKeyPrefix(containerId, "V1/B1/K1",
|
|
|
|
+ 0)) == 1);
|
|
|
|
+ Assert.assertTrue(
|
|
|
|
+ containerDbServiceProvider.getCountForForContainerKeyPrefix(
|
|
|
|
+ new ContainerKeyPrefix(containerId, "V1/B1/K2",
|
|
|
|
+ 0)) == 2);
|
|
|
|
+ Assert.assertTrue(
|
|
|
|
+ containerDbServiceProvider.getCountForForContainerKeyPrefix(
|
|
|
|
+ new ContainerKeyPrefix(containerId, "V1/B2/K3",
|
|
|
|
+ 0)) == 3);
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
@@ -109,11 +163,11 @@ public class TestContainerDBServiceProviderImpl {
|
|
long containerId = System.currentTimeMillis();
|
|
long containerId = System.currentTimeMillis();
|
|
|
|
|
|
containerDbServiceProvider.storeContainerKeyMapping(new
|
|
containerDbServiceProvider.storeContainerKeyMapping(new
|
|
- ContainerKeyPrefix(containerId, "V1/B1/K1"), 2);
|
|
|
|
|
|
+ ContainerKeyPrefix(containerId, "V2/B1/K1"), 2);
|
|
|
|
|
|
Integer count = containerDbServiceProvider.
|
|
Integer count = containerDbServiceProvider.
|
|
getCountForForContainerKeyPrefix(new ContainerKeyPrefix(containerId,
|
|
getCountForForContainerKeyPrefix(new ContainerKeyPrefix(containerId,
|
|
- "V1/B1/K1"));
|
|
|
|
|
|
+ "V2/B1/K1"));
|
|
assertTrue(count == 2);
|
|
assertTrue(count == 2);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -121,25 +175,32 @@ public class TestContainerDBServiceProviderImpl {
|
|
public void testGetKeyPrefixesForContainer() throws Exception {
|
|
public void testGetKeyPrefixesForContainer() throws Exception {
|
|
long containerId = System.currentTimeMillis();
|
|
long containerId = System.currentTimeMillis();
|
|
|
|
|
|
- containerDbServiceProvider.storeContainerKeyMapping(new
|
|
|
|
- ContainerKeyPrefix(containerId, "V1/B1/K1"), 1);
|
|
|
|
|
|
+ ContainerKeyPrefix containerKeyPrefix1 = new
|
|
|
|
+ ContainerKeyPrefix(containerId, "V3/B1/K1", 0);
|
|
|
|
+ containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix1,
|
|
|
|
+ 1);
|
|
|
|
|
|
- containerDbServiceProvider.storeContainerKeyMapping(new
|
|
|
|
- ContainerKeyPrefix(containerId, "V1/B1/K2"), 2);
|
|
|
|
|
|
+ ContainerKeyPrefix containerKeyPrefix2 = new ContainerKeyPrefix(
|
|
|
|
+ containerId, "V3/B1/K2", 0);
|
|
|
|
+ containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix2,
|
|
|
|
+ 2);
|
|
|
|
|
|
- long nextContainerId = System.currentTimeMillis();
|
|
|
|
- containerDbServiceProvider.storeContainerKeyMapping(new
|
|
|
|
- ContainerKeyPrefix(nextContainerId, "V1/B2/K1"), 3);
|
|
|
|
|
|
+ long nextContainerId = containerId + 1000L;
|
|
|
|
+ ContainerKeyPrefix containerKeyPrefix3 = new ContainerKeyPrefix(
|
|
|
|
+ nextContainerId, "V3/B2/K1", 0);
|
|
|
|
+ containerDbServiceProvider.storeContainerKeyMapping(containerKeyPrefix3,
|
|
|
|
+ 3);
|
|
|
|
|
|
- Map<String, Integer> keyPrefixMap = containerDbServiceProvider
|
|
|
|
- .getKeyPrefixesForContainer(containerId);
|
|
|
|
|
|
+ Map<ContainerKeyPrefix, Integer> keyPrefixMap =
|
|
|
|
+ containerDbServiceProvider.getKeyPrefixesForContainer(containerId);
|
|
assertTrue(keyPrefixMap.size() == 2);
|
|
assertTrue(keyPrefixMap.size() == 2);
|
|
- assertTrue(keyPrefixMap.get("V1/B1/K1") == 1);
|
|
|
|
- assertTrue(keyPrefixMap.get("V1/B1/K2") == 2);
|
|
|
|
|
|
|
|
- keyPrefixMap = containerDbServiceProvider
|
|
|
|
- .getKeyPrefixesForContainer(nextContainerId);
|
|
|
|
|
|
+ assertTrue(keyPrefixMap.get(containerKeyPrefix1) == 1);
|
|
|
|
+ assertTrue(keyPrefixMap.get(containerKeyPrefix2) == 2);
|
|
|
|
+
|
|
|
|
+ keyPrefixMap = containerDbServiceProvider.getKeyPrefixesForContainer(
|
|
|
|
+ nextContainerId);
|
|
assertTrue(keyPrefixMap.size() == 1);
|
|
assertTrue(keyPrefixMap.size() == 1);
|
|
- assertTrue(keyPrefixMap.get("V1/B2/K1") == 3);
|
|
|
|
|
|
+ assertTrue(keyPrefixMap.get(containerKeyPrefix3) == 3);
|
|
}
|
|
}
|
|
}
|
|
}
|