|
@@ -33,7 +33,6 @@ import com.amazonaws.services.dynamodbv2.document.Item;
|
|
|
import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
|
|
|
import com.amazonaws.services.dynamodbv2.document.Table;
|
|
|
import com.amazonaws.services.dynamodbv2.model.ListTagsOfResourceRequest;
|
|
|
-import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
|
|
|
import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
|
|
|
import com.amazonaws.services.dynamodbv2.model.TableDescription;
|
|
|
|
|
@@ -43,6 +42,7 @@ import org.apache.commons.collections.CollectionUtils;
|
|
|
import org.apache.commons.lang3.StringUtils;
|
|
|
import org.apache.hadoop.fs.contract.s3a.S3AContract;
|
|
|
import org.apache.hadoop.fs.s3a.Constants;
|
|
|
+import org.apache.hadoop.fs.s3a.S3ATestConstants;
|
|
|
import org.apache.hadoop.fs.s3a.Tristate;
|
|
|
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
@@ -64,6 +64,7 @@ import org.apache.hadoop.security.UserGroupInformation;
|
|
|
|
|
|
import static org.apache.hadoop.fs.s3a.Constants.*;
|
|
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
|
|
+import static org.apache.hadoop.fs.s3a.S3AUtils.clearBucketOption;
|
|
|
import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
|
|
|
import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.*;
|
|
|
import static org.apache.hadoop.test.LambdaTestUtils.*;
|
|
@@ -78,7 +79,15 @@ import static org.apache.hadoop.test.LambdaTestUtils.*;
|
|
|
*
|
|
|
* According to the base class, every test case will have independent contract
|
|
|
* to create a new {@link S3AFileSystem} instance and initializes it.
|
|
|
- * A table will be created and shared between the tests,
|
|
|
+ * A table will be created and shared between the tests; some tests also
|
|
|
+ * create their own.
|
|
|
+ *
|
|
|
+ * Important: Any new test which creates a table must do the following
|
|
|
+ * <ol>
|
|
|
+ * <li>Enable on-demand pricing.</li>
|
|
|
+ * <li>Always destroy the table, even if an assertion fails.</li>
|
|
|
+ * </ol>
|
|
|
+ * This is needed to avoid "leaking" DDB tables and running up bills.
|
|
|
*/
|
|
|
public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
|
|
@@ -121,7 +130,7 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
Assume.assumeTrue("Test DynamoDB table name should be set to run "
|
|
|
+ "integration tests.", testDynamoDBTableName != null);
|
|
|
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, testDynamoDBTableName);
|
|
|
-
|
|
|
+ enableOnDemand(conf);
|
|
|
s3AContract = new S3AContract(conf);
|
|
|
s3AContract.init();
|
|
|
|
|
@@ -141,36 +150,40 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-
|
|
|
@BeforeClass
|
|
|
public static void beforeClassSetup() throws IOException {
|
|
|
Configuration conf = prepareTestConfiguration(new Configuration());
|
|
|
assumeThatDynamoMetadataStoreImpl(conf);
|
|
|
// S3GUARD_DDB_TEST_TABLE_NAME_KEY and S3GUARD_DDB_TABLE_NAME_KEY should
|
|
|
// be configured to use this test.
|
|
|
- testDynamoDBTableName = conf.get(S3GUARD_DDB_TEST_TABLE_NAME_KEY);
|
|
|
+ testDynamoDBTableName = conf.get(
|
|
|
+ S3ATestConstants.S3GUARD_DDB_TEST_TABLE_NAME_KEY);
|
|
|
String dynamoDbTableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY);
|
|
|
- Assume.assumeTrue("No DynamoDB table name configured", !StringUtils
|
|
|
- .isEmpty(dynamoDbTableName));
|
|
|
+ Assume.assumeTrue("No DynamoDB table name configured",
|
|
|
+ !StringUtils.isEmpty(dynamoDbTableName));
|
|
|
|
|
|
// We should assert that the table name is configured, so the test should
|
|
|
// fail if it's not configured.
|
|
|
- assertTrue("Test DynamoDB table name '"
|
|
|
- + S3GUARD_DDB_TEST_TABLE_NAME_KEY + "' should be set to run "
|
|
|
- + "integration tests.", testDynamoDBTableName != null);
|
|
|
+ assertNotNull("Test DynamoDB table name '"
|
|
|
+ + S3ATestConstants.S3GUARD_DDB_TEST_TABLE_NAME_KEY + "'"
|
|
|
+ + " should be set to run integration tests.",
|
|
|
+ testDynamoDBTableName);
|
|
|
|
|
|
// We should assert that the test table is not the same as the production
|
|
|
// table, as the test table could be modified and destroyed multiple
|
|
|
// times during the test.
|
|
|
- assertTrue("Test DynamoDB table name: '"
|
|
|
- + S3GUARD_DDB_TEST_TABLE_NAME_KEY + "' and production table name: '"
|
|
|
- + S3GUARD_DDB_TABLE_NAME_KEY + "' can not be the same.",
|
|
|
- !conf.get(S3GUARD_DDB_TABLE_NAME_KEY).equals(testDynamoDBTableName));
|
|
|
+ assertNotEquals("Test DynamoDB table name: "
|
|
|
+ + "'" + S3ATestConstants.S3GUARD_DDB_TEST_TABLE_NAME_KEY + "'"
|
|
|
+ + " and production table name: "
|
|
|
+ + "'" + S3GUARD_DDB_TABLE_NAME_KEY + "' can not be the same.",
|
|
|
+ testDynamoDBTableName, conf.get(S3GUARD_DDB_TABLE_NAME_KEY));
|
|
|
|
|
|
// We can use that table in the test if these assertions are valid
|
|
|
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, testDynamoDBTableName);
|
|
|
|
|
|
LOG.debug("Creating static ddbms which will be shared between tests.");
|
|
|
+ enableOnDemand(conf);
|
|
|
+
|
|
|
ddbmsStatic = new DynamoDBMetadataStore();
|
|
|
ddbmsStatic.initialize(conf);
|
|
|
}
|
|
@@ -198,18 +211,23 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
|
|
|
@Override
|
|
|
public void tearDown() throws Exception {
|
|
|
- LOG.info("Removing data from ddbms table in teardown.");
|
|
|
- // The following is a way to be sure the table will be cleared and there
|
|
|
- // will be no leftovers after the test.
|
|
|
- PathMetadata meta = ddbmsStatic.get(strToPath("/"));
|
|
|
- if (meta != null){
|
|
|
- for (DescendantsIterator desc = new DescendantsIterator(ddbmsStatic, meta);
|
|
|
- desc.hasNext();) {
|
|
|
- ddbmsStatic.forgetMetadata(desc.next().getPath());
|
|
|
+ try {
|
|
|
+ if (ddbmsStatic != null) {
|
|
|
+ LOG.info("Removing data from ddbms table in teardown.");
|
|
|
+ // The following is a way to be sure the table will be cleared and there
|
|
|
+ // will be no leftovers after the test.
|
|
|
+ PathMetadata meta = ddbmsStatic.get(strToPath("/"));
|
|
|
+ if (meta != null){
|
|
|
+ for (DescendantsIterator desc =
|
|
|
+ new DescendantsIterator(ddbmsStatic, meta);
|
|
|
+ desc.hasNext();) {
|
|
|
+ ddbmsStatic.forgetMetadata(desc.next().getPath());
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
+ } catch (IOException ignored) {
|
|
|
}
|
|
|
-
|
|
|
- fileSystem.close();
|
|
|
+ IOUtils.cleanupWithLogger(LOG, fileSystem);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -263,6 +281,29 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
return this.fileSystem;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Force the configuration into DDB on demand, so that
|
|
|
+ * even if a test bucket isn't cleaned up, the cost is $0.
|
|
|
+ * @param conf configuration to patch.
|
|
|
+ */
|
|
|
+ public static void enableOnDemand(Configuration conf) {
|
|
|
+ conf.setInt(S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY, 0);
|
|
|
+ conf.setInt(S3GUARD_DDB_TABLE_CAPACITY_READ_KEY, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the configuration needed to create a table; extracts
|
|
|
+ * it from the filesystem then always patches it to be on demand.
|
|
|
+ * Why the patch? It means even if a cached FS has brought in
|
|
|
+ * some provisioned values, they get reset.
|
|
|
+ * @return a new configuration
|
|
|
+ */
|
|
|
+ private Configuration getTableCreationConfig() {
|
|
|
+ Configuration conf = new Configuration(getFileSystem().getConf());
|
|
|
+ enableOnDemand(conf);
|
|
|
+ return conf;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* This tests that after initialize() using an S3AFileSystem object, the
|
|
|
* instance should have been initialized successfully, and tables are ACTIVE.
|
|
@@ -272,9 +313,11 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
final S3AFileSystem s3afs = this.fileSystem;
|
|
|
final String tableName =
|
|
|
getTestTableName("testInitialize");
|
|
|
- final Configuration conf = s3afs.getConf();
|
|
|
+ Configuration conf = getFileSystem().getConf();
|
|
|
+ enableOnDemand(conf);
|
|
|
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
|
|
|
- try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
|
|
|
+ DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
|
|
|
+ try {
|
|
|
ddbms.initialize(s3afs);
|
|
|
verifyTableInitialized(tableName, ddbms.getDynamoDB());
|
|
|
assertNotNull(ddbms.getTable());
|
|
@@ -285,7 +328,9 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
" region as S3 bucket",
|
|
|
expectedRegion,
|
|
|
ddbms.getRegion());
|
|
|
+ } finally {
|
|
|
ddbms.destroy();
|
|
|
+ ddbms.close();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -297,7 +342,7 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
public void testInitializeWithConfiguration() throws IOException {
|
|
|
final String tableName =
|
|
|
getTestTableName("testInitializeWithConfiguration");
|
|
|
- final Configuration conf = getFileSystem().getConf();
|
|
|
+ final Configuration conf = getTableCreationConfig();
|
|
|
conf.unset(S3GUARD_DDB_TABLE_NAME_KEY);
|
|
|
String savedRegion = conf.get(S3GUARD_DDB_REGION_KEY,
|
|
|
getFileSystem().getBucketLocation());
|
|
@@ -316,7 +361,8 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
}
|
|
|
// config region
|
|
|
conf.set(S3GUARD_DDB_REGION_KEY, savedRegion);
|
|
|
- try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
|
|
|
+ DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
|
|
|
+ try {
|
|
|
ddbms.initialize(conf);
|
|
|
verifyTableInitialized(tableName, ddbms.getDynamoDB());
|
|
|
assertNotNull(ddbms.getTable());
|
|
@@ -324,7 +370,9 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
assertEquals("Unexpected key schema found!",
|
|
|
keySchema(),
|
|
|
ddbms.getTable().describe().getKeySchema());
|
|
|
+ } finally {
|
|
|
ddbms.destroy();
|
|
|
+ ddbms.close();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -434,13 +482,14 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
@Test
|
|
|
public void testTableVersionRequired() throws Exception {
|
|
|
String tableName = getTestTableName("testTableVersionRequired");
|
|
|
- Configuration conf = getFileSystem().getConf();
|
|
|
+ Configuration conf = getTableCreationConfig();
|
|
|
int maxRetries = conf.getInt(S3GUARD_DDB_MAX_RETRIES,
|
|
|
S3GUARD_DDB_MAX_RETRIES_DEFAULT);
|
|
|
conf.setInt(S3GUARD_DDB_MAX_RETRIES, 3);
|
|
|
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
|
|
|
|
|
|
- try(DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
|
|
|
+ DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
|
|
|
+ try {
|
|
|
ddbms.initialize(conf);
|
|
|
Table table = verifyTableInitialized(tableName, ddbms.getDynamoDB());
|
|
|
table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
|
|
@@ -450,7 +499,9 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
() -> ddbms.initTable());
|
|
|
|
|
|
conf.setInt(S3GUARD_DDB_MAX_RETRIES, maxRetries);
|
|
|
+ } finally {
|
|
|
ddbms.destroy();
|
|
|
+ ddbms.close();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -461,10 +512,11 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
@Test
|
|
|
public void testTableVersionMismatch() throws Exception {
|
|
|
String tableName = getTestTableName("testTableVersionMismatch");
|
|
|
- Configuration conf = getFileSystem().getConf();
|
|
|
+ Configuration conf = getTableCreationConfig();
|
|
|
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
|
|
|
|
|
|
- try(DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
|
|
|
+ DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
|
|
|
+ try {
|
|
|
ddbms.initialize(conf);
|
|
|
Table table = verifyTableInitialized(tableName, ddbms.getDynamoDB());
|
|
|
table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
|
|
@@ -474,7 +526,9 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
// create existing table
|
|
|
intercept(IOException.class, E_INCOMPATIBLE_VERSION,
|
|
|
() -> ddbms.initTable());
|
|
|
+ } finally {
|
|
|
ddbms.destroy();
|
|
|
+ ddbms.close();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -491,10 +545,18 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
getTestTableName("testFailNonexistentTable");
|
|
|
final S3AFileSystem s3afs = getFileSystem();
|
|
|
final Configuration conf = s3afs.getConf();
|
|
|
+ enableOnDemand(conf);
|
|
|
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
|
|
|
+ String b = fsUri.getHost();
|
|
|
+ clearBucketOption(conf, b, S3GUARD_DDB_TABLE_CREATE_KEY);
|
|
|
+ clearBucketOption(conf, b, S3_METADATA_STORE_IMPL);
|
|
|
+ clearBucketOption(conf, b, S3GUARD_DDB_TABLE_NAME_KEY);
|
|
|
conf.unset(S3GUARD_DDB_TABLE_CREATE_KEY);
|
|
|
try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
|
|
|
ddbms.initialize(s3afs);
|
|
|
+ // if an exception was not raised, a table was created.
|
|
|
+ // So destroy it before failing.
|
|
|
+ ddbms.destroy();
|
|
|
fail("Should have failed as table does not exist and table auto-creation"
|
|
|
+ " is disabled");
|
|
|
} catch (IOException ignored) {
|
|
@@ -606,31 +668,36 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
public void testProvisionTable() throws Exception {
|
|
|
final String tableName
|
|
|
= getTestTableName("testProvisionTable-" + UUID.randomUUID());
|
|
|
- Configuration conf = getFileSystem().getConf();
|
|
|
+ final Configuration conf = getTableCreationConfig();
|
|
|
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
|
|
|
-
|
|
|
- try(DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
|
|
|
+ conf.setInt(S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY, 2);
|
|
|
+ conf.setInt(S3GUARD_DDB_TABLE_CAPACITY_READ_KEY, 2);
|
|
|
+ DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
|
|
|
+ try {
|
|
|
ddbms.initialize(conf);
|
|
|
DynamoDB dynamoDB = ddbms.getDynamoDB();
|
|
|
- final ProvisionedThroughputDescription oldProvision =
|
|
|
- dynamoDB.getTable(tableName).describe().getProvisionedThroughput();
|
|
|
- ddbms.provisionTable(oldProvision.getReadCapacityUnits() * 2,
|
|
|
- oldProvision.getWriteCapacityUnits() * 2);
|
|
|
+ final DDBCapacities oldProvision = DDBCapacities.extractCapacities(
|
|
|
+ dynamoDB.getTable(tableName).describe().getProvisionedThroughput());
|
|
|
+ Assume.assumeFalse("Table is on-demand", oldProvision.isOnDemandTable());
|
|
|
+ long desiredReadCapacity = oldProvision.getRead() - 1;
|
|
|
+ long desiredWriteCapacity = oldProvision.getWrite() - 1;
|
|
|
+ ddbms.provisionTable(desiredReadCapacity,
|
|
|
+ desiredWriteCapacity);
|
|
|
ddbms.initTable();
|
|
|
// we have to wait until the provisioning settings are applied,
|
|
|
// so until the table is ACTIVE again and not in UPDATING
|
|
|
ddbms.getTable().waitForActive();
|
|
|
- final ProvisionedThroughputDescription newProvision =
|
|
|
- dynamoDB.getTable(tableName).describe().getProvisionedThroughput();
|
|
|
- LOG.info("Old provision = {}, new provision = {}", oldProvision,
|
|
|
- newProvision);
|
|
|
+ final DDBCapacities newProvision = DDBCapacities.extractCapacities(
|
|
|
+ dynamoDB.getTable(tableName).describe().getProvisionedThroughput());
|
|
|
assertEquals("Check newly provisioned table read capacity units.",
|
|
|
- oldProvision.getReadCapacityUnits() * 2,
|
|
|
- newProvision.getReadCapacityUnits().longValue());
|
|
|
+ desiredReadCapacity,
|
|
|
+ newProvision.getRead());
|
|
|
assertEquals("Check newly provisioned table write capacity units.",
|
|
|
- oldProvision.getWriteCapacityUnits() * 2,
|
|
|
- newProvision.getWriteCapacityUnits().longValue());
|
|
|
+ desiredWriteCapacity,
|
|
|
+ newProvision.getWrite());
|
|
|
+ } finally {
|
|
|
ddbms.destroy();
|
|
|
+ ddbms.close();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -639,9 +706,11 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
final String tableName = getTestTableName("testDeleteTable");
|
|
|
Path testPath = new Path(new Path(fsUri), "/" + tableName);
|
|
|
final S3AFileSystem s3afs = getFileSystem();
|
|
|
- final Configuration conf = s3afs.getConf();
|
|
|
+ final Configuration conf = getTableCreationConfig();
|
|
|
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
|
|
|
- try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
|
|
|
+ enableOnDemand(conf);
|
|
|
+ DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
|
|
|
+ try {
|
|
|
ddbms.initialize(s3afs);
|
|
|
// we can list the empty table
|
|
|
ddbms.listChildren(testPath);
|
|
@@ -649,23 +718,22 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
ddbms.destroy();
|
|
|
verifyTableNotExist(tableName, dynamoDB);
|
|
|
|
|
|
- // delete table once more; be ResourceNotFoundException swallowed silently
|
|
|
+ // delete table once more; the ResourceNotFoundException swallowed
|
|
|
+ // silently
|
|
|
ddbms.destroy();
|
|
|
verifyTableNotExist(tableName, dynamoDB);
|
|
|
- try {
|
|
|
- // we can no longer list the destroyed table
|
|
|
- ddbms.listChildren(testPath);
|
|
|
- fail("Should have failed after the table is destroyed!");
|
|
|
- } catch (IOException ignored) {
|
|
|
- }
|
|
|
+ intercept(IOException.class, "",
|
|
|
+ "Should have failed after the table is destroyed!",
|
|
|
+ () -> ddbms.listChildren(testPath));
|
|
|
+ } finally {
|
|
|
ddbms.destroy();
|
|
|
+ ddbms.close();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testTableTagging() throws IOException {
|
|
|
- final Configuration conf = getFileSystem().getConf();
|
|
|
-
|
|
|
+ final Configuration conf = getTableCreationConfig();
|
|
|
// clear all table tagging config before this test
|
|
|
conf.getPropsWithPrefix(S3GUARD_DDB_TABLE_TAG).keySet().forEach(
|
|
|
propKey -> conf.unset(S3GUARD_DDB_TABLE_TAG + propKey)
|
|
@@ -683,7 +751,8 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
conf.set(S3GUARD_DDB_TABLE_TAG + tagEntry.getKey(), tagEntry.getValue());
|
|
|
}
|
|
|
|
|
|
- try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
|
|
|
+ DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
|
|
|
+ try {
|
|
|
ddbms.initialize(conf);
|
|
|
assertNotNull(ddbms.getTable());
|
|
|
assertEquals(tableName, ddbms.getTable().getTableName());
|
|
@@ -696,6 +765,9 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
|
|
|
for (Tag tag : tags) {
|
|
|
Assert.assertEquals(tagMap.get(tag.getKey()), tag.getValue());
|
|
|
}
|
|
|
+ } finally {
|
|
|
+ ddbms.destroy();
|
|
|
+ ddbms.close();
|
|
|
}
|
|
|
}
|
|
|
|