Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@ public class HddsVolume extends StorageVolume {
private File dbParentDir;
private File deletedContainerDir;
private AtomicBoolean dbLoaded = new AtomicBoolean(false);
private final AtomicBoolean dbLoadFailure = new AtomicBoolean(false);

/**
* Builder for HddsVolume.
Expand Down Expand Up @@ -257,6 +258,11 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused)
VolumeCheckResult result = super.check(unused);

DatanodeConfiguration df = getConf().getObject(DatanodeConfiguration.class);
if (isDbLoadFailure()) {
LOG.warn("Volume {} failed to access RocksDB: RocksDB parent directory is null, " +
"the volume might not have been loaded properly.", getStorageDir());
return VolumeCheckResult.FAILED;
}
if (result != VolumeCheckResult.HEALTHY ||
!df.getContainerSchemaV3Enabled() || !isDbLoaded()) {
return result;
Expand Down Expand Up @@ -313,6 +319,11 @@ public File getDbParentDir() {
return this.dbParentDir;
}

@VisibleForTesting
public void setDbParentDir(File dbParentDir) {
this.dbParentDir = dbParentDir;
}

public File getDeletedContainerDir() {
return this.deletedContainerDir;
}
Expand All @@ -326,6 +337,10 @@ public boolean isDbLoaded() {
return dbLoaded.get();
}

public boolean isDbLoadFailure() {
return dbLoadFailure.get();
}

public void loadDbStore(boolean readOnly) throws IOException {
// DN startup for the first time, not registered yet,
// so the DbVolume is not formatted.
Expand Down Expand Up @@ -363,7 +378,8 @@ public void loadDbStore(boolean readOnly) throws IOException {
String containerDBPath = containerDBFile.getAbsolutePath();
try {
initPerDiskDBStore(containerDBPath, getConf(), readOnly);
} catch (IOException e) {
} catch (Throwable e) {
dbLoadFailure.set(true);
Copy link
Contributor

Choose a reason for hiding this comment

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

@xichen01 , thanks for reporting this. Can you share more detail about which operation will cause the DB load failure and throw the IOException, besides the known initPerDiskDBStore? Is there any exception stack can be shared?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not sure of the root cause of the DB loading issue, but we found a very large container.db in the system home directory, as well as error logs from HddsVolume initing.

logs from HddsVolume initing
image

container.db in the system home
image

Copy link
Contributor Author

Choose a reason for hiding this comment

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

  1. Manually destroy the RocksDB file. 2. Then restart DN and write data. can reproduce a similar problem.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

 ERROR org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer: Load db store for HddsVolume /xxx/xxx/xxx/ozone/hdds failed
java.io.IOException: Can't init db instance under path /xxx/xxxx/xxx/ozone/hdds/CID-xxx/DS-xxx/container.db for volume DS-xxx
        at org.apache.hadoop.ozone.container.common.volume.HddsVolume.loadDbStore(HddsVolume.java:235)
        at org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil.loadAllHddsVolumeDbStore(HddsVolumeUtil.java:99)
        at org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer.<init>(OzoneContainer.java:146)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.<init>(DatanodeStateMachine.java:153)
        at org.apache.hadoop.ozone.HddsDatanodeService.start(HddsDatanodeService.java:295)
        at org.apache.hadoop.ozone.HddsDatanodeService.start(HddsDatanodeService.java:227)
        at org.apache.hadoop.ozone.HddsDatanodeService.call(HddsDatanodeService.java:195)
        at org.apache.hadoop.ozone.HddsDatanodeService.call(HddsDatanodeService.java:104)
        at picocli.CommandLine.executeUserObject(CommandLine.java:1953)
        at picocli.CommandLine.access$1300(CommandLine.java:145)
        at picocli.CommandLine$RunLast.executeUserObjectOfLastSubcommandWithSameParent(CommandLine.java:2352)
        at picocli.CommandLine$RunLast.handle(CommandLine.java:2346)
        at picocli.CommandLine$RunLast.handle(CommandLine.java:2311)
        at picocli.CommandLine$AbstractParseResultHandler.handleParseResult(CommandLine.java:2172)
        at picocli.CommandLine.parseWithHandlers(CommandLine.java:2550)
        at picocli.CommandLine.parseWithHandler(CommandLine.java:2485)
        at org.apache.hadoop.hdds.cli.GenericCli.execute(GenericCli.java:96)
        at org.apache.hadoop.hdds.cli.GenericCli.run(GenericCli.java:87)
        at org.apache.hadoop.ozone.HddsDatanodeService.main(HddsDatanodeService.java:178)
Caused by: java.io.IOException: Failed init RocksDB, db path : /xxx/xxx/xxx/ozone/hdds/CID-xxx/DS-xxx/container.db, exception :org.rocksdb.RocksDBException CURRENT file does not end with newline
        at org.apache.hadoop.hdds.utils.db.RDBStore.<init>(RDBStore.java:130)
        at org.apache.hadoop.hdds.utils.db.DBStoreBuilder.build(DBStoreBuilder.java:191)
        at org.apache.hadoop.ozone.container.metadata.AbstractDatanodeStore.start(AbstractDatanodeStore.java:146)
        at org.apache.hadoop.ozone.container.metadata.AbstractDatanodeStore.<init>(AbstractDatanodeStore.java:99)
        at org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaThreeImpl.<init>(DatanodeStoreSchemaThreeImpl.java:54)
        at org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils.getUncachedDatanodeStore(BlockUtils.java:84)
        at org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil.initPerDiskDBStore(HddsVolumeUtil.java:79)
        at org.apache.hadoop.ozone.container.common.volume.HddsVolume.loadDbStore(HddsVolume.java:232)
        ... 18 more

Copy link
Contributor

Choose a reason for hiding this comment

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

I think we need to root cause this issue. Could you kindly check the related DN logs to see if there is any related suspicious logs? If we don't have the root cause, then this extra IOException catch may not enough.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not found others related log which can help to locate the root cause, this may be a legacy issue that may have existed when the cluster was created, but with this PR, we could have found this issue faster.

Copy link
Contributor

Choose a reason for hiding this comment

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

I have observed another impact when volume metadata path is removed incorrectly in test env. But at that point of time, container was getting loaded having NPE in other flow.
NPE was handled with #5921 avoiding loading of container.
Since it was test script issue where cleanup was not proper, referring similar case.

Copy link
Contributor

Choose a reason for hiding this comment

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

@xichen01 , in case we don't know the exact case of DB loading failure, let's change this catch from IOException to Throwable.

Copy link
Contributor

Choose a reason for hiding this comment

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

@xichen01 , could you please address my above comment and provide a new patch?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

OK, I will

throw new IOException("Can't init db instance under path "
+ containerDBPath + " for volume " + getStorageID(), e);
}
Expand Down Expand Up @@ -417,9 +433,11 @@ public void createDbStore(MutableVolumeSet dbVolumeSet) throws IOException {
try {
HddsVolumeUtil.initPerDiskDBStore(containerDBPath, getConf(), false);
dbLoaded.set(true);
dbLoadFailure.set(false);
LOG.info("SchemaV3 db is created and loaded at {} for volume {}",
containerDBPath, getStorageID());
} catch (IOException e) {
dbLoadFailure.set(true);
String errMsg = "Can't create db instance under path "
+ containerDBPath + " for volume " + getStorageID();
LOG.error(errMsg, e);
Expand Down Expand Up @@ -448,6 +466,7 @@ private void closeDbStore() {
.getAbsolutePath();
DatanodeStoreCache.getInstance().removeDB(containerDBPath);
dbLoaded.set(false);
dbLoadFailure.set(false);
LOG.info("SchemaV3 db is stopped at {} for volume {}", containerDBPath,
getStorageID());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -442,12 +442,20 @@ public Map<StorageType, List<StorageVolume>> getVolumeStateMap() {
public boolean hasEnoughVolumes() {
// Max number of bad volumes allowed, should have at least
// 1 good volume
boolean hasEnoughVolumes;
if (maxVolumeFailuresTolerated ==
StorageVolumeChecker.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
return getVolumesList().size() >= 1;
hasEnoughVolumes = getVolumesList().size() >= 1;
} else {
return getFailedVolumesList().size() <= maxVolumeFailuresTolerated;
hasEnoughVolumes = getFailedVolumesList().size() <= maxVolumeFailuresTolerated;
}
if (!hasEnoughVolumes) {
LOG.error("Not enough volumes in MutableVolumeSet. DatanodeUUID: {}, VolumeType: {}, " +
"MaxVolumeFailuresTolerated: {}, ActiveVolumes: {}, FailedVolumes: {}",
datanodeUuid, volumeType, maxVolumeFailuresTolerated,
getVolumesList().size(), getFailedVolumesList().size());
}
return hasEnoughVolumes;
}

public StorageLocationReport[] getStorageReport() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -124,15 +124,13 @@ private static String getContainerSubDirectory(long containerId) {
*/
public static File getContainerDBFile(KeyValueContainerData containerData) {
if (containerData.hasSchema(OzoneConsts.SCHEMA_V3)) {
Preconditions.checkNotNull(containerData.getVolume().getDbParentDir(), "Base Directory cannot be null");
return new File(containerData.getVolume().getDbParentDir(),
OzoneConsts.CONTAINER_DB_NAME);
}
return getContainerDBFile(containerData.getMetadataPath(), containerData);
}

public static File getContainerDBFile(String baseDir,
KeyValueContainerData containerData) {
return new File(baseDir, containerData.getContainerID() +
Preconditions.checkNotNull(containerData.getMetadataPath(), "Metadata Directory cannot be null");
return new File(containerData.getMetadataPath(), containerData.getContainerID() +
OzoneConsts.DN_CONTAINER_DB);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext.Op;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext.WriteChunkStage;
import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
Expand All @@ -73,6 +74,7 @@

import java.io.File;
import java.io.IOException;
import java.nio.file.Path;
import java.time.Duration;
import java.util.Collections;
import java.util.HashMap;
Expand Down Expand Up @@ -103,6 +105,9 @@
* Test-cases to verify the functionality of HddsDispatcher.
*/
public class TestHddsDispatcher {
@TempDir
private Path tempDir;

private static final Logger LOG = LoggerFactory.getLogger(
TestHddsDispatcher.class);
@TempDir
Expand Down Expand Up @@ -133,6 +138,8 @@ public void testContainerCloseActionWhenFull(
(long) StorageUnit.GB.toBytes(1), UUID.randomUUID().toString(),
dd.getUuidString());
Container container = new KeyValueContainer(containerData, conf);
StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())
.forEach(hddsVolume -> hddsVolume.setDbParentDir(tempDir.toFile()));
container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(),
scmId.toString());
containerSet.addContainer(container);
Expand Down Expand Up @@ -267,6 +274,8 @@ public void testContainerCloseActionWhenVolumeFull(
50, UUID.randomUUID().toString(),
dd.getUuidString());
Container container = new KeyValueContainer(containerData, conf);
StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())
.forEach(hddsVolume -> hddsVolume.setDbParentDir(tempDir.toFile()));
container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(),
scmId.toString());
containerSet.addContainer(container);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
import org.apache.hadoop.ozone.container.common.volume.DbVolume;
Expand All @@ -43,7 +44,13 @@
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrowsExactly;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.mockStatic;

import org.mockito.MockedStatic;
import org.mockito.Mockito;


/**
* Test for {@link HddsVolumeUtil}.
Expand Down Expand Up @@ -95,6 +102,34 @@ public void teardown() {
dbVolumeSet.shutdown();
}

@Test
public void testLoadHDDVolumeWithInitDBException()
throws Exception {
// Create db instances for all HDDsVolumes.
for (HddsVolume hddsVolume : StorageVolumeUtil.getHddsVolumesList(
hddsVolumeSet.getVolumesList())) {
hddsVolume.format(clusterId);
hddsVolume.createWorkingDir(clusterId, null);
}

try (MockedStatic<HddsVolumeUtil> mocked = mockStatic(HddsVolumeUtil.class, Mockito.CALLS_REAL_METHODS)) {
// Simulating the init DB Exception
mocked.when(() -> HddsVolumeUtil.initPerDiskDBStore(Mockito.anyString(), Mockito.any(), Mockito.anyBoolean()))
.thenThrow(new IOException("Mocked Exception"));

reinitVolumes();
for (HddsVolume hddsVolume : StorageVolumeUtil.getHddsVolumesList(
hddsVolumeSet.getVolumesList())) {
assertThrowsExactly(IOException.class, () -> hddsVolume.loadDbStore(true));
// If the Volume init DB is abnormal, the Volume should be recognized as a failed Volume
assertEquals(VolumeCheckResult.FAILED, hddsVolume.check(false));
assertTrue(hddsVolume.isDbLoadFailure());
assertFalse(hddsVolume.isDbLoaded());
}
}

}

@Test
public void testLoadAllHddsVolumeDbStoreWithoutDbVolumes()
throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
Expand All @@ -45,6 +46,7 @@
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
Expand Down Expand Up @@ -76,6 +78,9 @@
*/
@Timeout(30)
public class TestVolumeSetDiskChecks {
@TempDir
private Path tempDir;

public static final Logger LOG = LoggerFactory.getLogger(
TestVolumeSetDiskChecks.class);
@TempDir
Expand Down Expand Up @@ -302,11 +307,15 @@ public void testVolumeFailure() throws IOException {
dummyChecker);

KeyValueContainer container = new KeyValueContainer(data, conf);
StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())
.forEach(hddsVolume -> hddsVolume.setDbParentDir(tempDir.toFile()));
container.create(volumeSet,
new RoundRobinVolumeChoosingPolicy(), UUID.randomUUID().toString());
conSet.addContainer(container);

KeyValueContainer container1 = new KeyValueContainer(data1, conf);
StorageVolumeUtil.getHddsVolumesList(volumeSet1.getVolumesList())
.forEach(hddsVolume -> hddsVolume.setDbParentDir(tempDir.toFile()));
container1.create(volumeSet1,
new RoundRobinVolumeChoosingPolicy(), UUID.randomUUID().toString());
conSet.addContainer(container1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
Expand Down Expand Up @@ -143,6 +144,8 @@ public void init(boolean isZeroCopy) throws Exception {
ContainerLayoutVersion.FILE_PER_BLOCK, GB, UUID.randomUUID().toString(),
datanode.getUuidString());
KeyValueContainer container = new KeyValueContainer(data, conf);
StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())
.forEach(hddsVolume -> hddsVolume.setDbParentDir(tempDir.toFile()));
container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(),
"test-replication");
containerSet.addContainer(container);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine;
import org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask;
import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.replication.ContainerImporter;
import org.apache.hadoop.ozone.container.replication.ContainerReplicationSource;
Expand Down Expand Up @@ -532,6 +533,8 @@ public void restartDatanode(int expectedMlv, boolean exactMatch)

// Start new datanode with the same configuration.
dsm = new DatanodeStateMachine(dd, conf);
StorageVolumeUtil.getHddsVolumesList(dsm.getContainer().getVolumeSet().getVolumesList())
.forEach(hddsVolume -> hddsVolume.setDbParentDir(tempFolder.toFile()));
int mlv = dsm.getLayoutVersionManager().getMetadataLayoutVersion();
if (exactMatch) {
assertEquals(expectedMlv, mlv);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -184,9 +184,9 @@ public void testDeletedContainersClearedOnStartup() throws Exception {
ozoneConf.setBoolean(OzoneConfigKeys.HDDS_CONTAINER_RATIS_IPC_RANDOM_PORT,
true);
ozoneConf.setFromObject(new ReplicationConfig().setPort(0));
OzoneContainer ozoneContainer = createVolume(ozoneConf);
try (EndpointStateMachine rpcEndPoint = createEndpoint(ozoneConf,
serverAddress, 1000)) {
OzoneContainer ozoneContainer = createVolume(ozoneConf);
HddsVolume hddsVolume = (HddsVolume) ozoneContainer.getVolumeSet()
.getVolumesList().get(0);
KeyValueContainer kvContainer = addContainer(ozoneConf, hddsVolume);
Expand All @@ -212,6 +212,8 @@ public void testDeletedContainersClearedOnStartup() throws Exception {
hddsVolume.getDeletedContainerDir().listFiles();
assertNotNull(leftoverContainers);
assertEquals(0, leftoverContainers.length);
} finally {
ozoneContainer.stop();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.hadoop.ozone.container.metrics;

import java.io.File;
import java.nio.file.Path;
import java.util.List;
import java.util.Map;
import java.util.UUID;
Expand Down Expand Up @@ -59,12 +60,15 @@

import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.api.io.TempDir;

/**
* Test for metrics published by storage containers.
*/
@Timeout(300)
public class TestContainerMetrics {
@TempDir
private Path tempDir;

@Test
public void testContainerMetrics() throws Exception {
Expand Down Expand Up @@ -105,6 +109,8 @@ public void testContainerMetrics() throws Exception {
}
HddsDispatcher dispatcher = new HddsDispatcher(conf, containerSet,
volumeSet, handlers, context, metrics, null);
StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())
.forEach(hddsVolume -> hddsVolume.setDbParentDir(tempDir.toFile()));
dispatcher.setClusterId(UUID.randomUUID().toString());

server = new XceiverServerGrpc(datanodeDetails, conf, dispatcher, null);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,11 +30,13 @@
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.api.io.TempDir;

import java.io.File;
import java.nio.file.Path;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
Expand All @@ -55,6 +57,8 @@
*/
@Timeout(300)
public class TestOzoneContainer {
@TempDir
private Path tempDir;

@Test
public void testCreateOzoneContainer(
Expand All @@ -75,6 +79,8 @@ public void testCreateOzoneContainer(
DatanodeDetails datanodeDetails = randomDatanodeDetails();
container = ContainerTestUtils
.getOzoneContainer(datanodeDetails, conf);
StorageVolumeUtil.getHddsVolumesList(container.getVolumeSet().getVolumesList())
.forEach(hddsVolume -> hddsVolume.setDbParentDir(tempDir.toFile()));
//Set clusterId and manually start ozone container.
container.start(UUID.randomUUID().toString());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,8 @@
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
import org.apache.hadoop.ozone.container.replication.SimpleContainerDownloader;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
Expand Down Expand Up @@ -303,6 +305,9 @@ private OzoneContainer createAndStartOzoneContainerInstance() {
StateContext stateContext = ContainerTestUtils.getMockContext(dn, conf);
container = new OzoneContainer(
dn, conf, stateContext, caClient, keyClient);
MutableVolumeSet volumeSet = container.getVolumeSet();
StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())
.forEach(hddsVolume -> hddsVolume.setDbParentDir(tempFolder.toFile()));
container.start(clusterID);
} catch (Throwable e) {
if (container != null) {
Expand Down
Loading