This is an automated email from the ASF dual-hosted git repository.
gjahad pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git
The following commit(s) were added to refs/heads/master by this push:
new 1ae0401b1a HDDS-8681. [SNAPSHOT] Intermittent failure in
TestOMDbCheckpointServlet#testWriteDbDataToStream (#4950)
1ae0401b1a is described below
commit 1ae0401b1ac662c021b37b68ef46f4b9a0909890
Author: GeorgeJahad <[email protected]>
AuthorDate: Thu Jun 29 15:05:15 2023 -0700
HDDS-8681. [SNAPSHOT] Intermittent failure in
TestOMDbCheckpointServlet#testWriteDbDataToStream (#4950)
Co-authored-by: George Jahad <[email protected]>
Co-authored-by: prashantpogde <[email protected]>
Co-authored-by: Siyao Meng <[email protected]>
Co-authored-by: Hemant Kumar <[email protected]>
---
.../hadoop/hdds/utils/DBCheckpointServlet.java | 70 +++++++++++++++-------
.../hdds/scm/TestSCMDbCheckpointServlet.java | 3 +
.../hadoop/ozone/om/TestOMDbCheckpointServlet.java | 40 ++++++++++---
.../hadoop/ozone/om/OMDBCheckpointServlet.java | 6 +-
4 files changed, 85 insertions(+), 34 deletions(-)
diff --git
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java
index 7d19fc4568..e5e6682aa9 100644
---
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java
+++
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java
@@ -48,6 +48,7 @@ import static
org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_FL
import static
org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST;
import static org.apache.hadoop.ozone.OzoneConsts.ROCKSDB_SST_SUFFIX;
+import org.apache.hadoop.ozone.lock.BootstrapStateHandler;
import org.apache.hadoop.security.UserGroupInformation;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -55,7 +56,8 @@ import org.slf4j.LoggerFactory;
/**
* Provides the current checkpoint Snapshot of the OM/SCM DB. (tar)
*/
-public class DBCheckpointServlet extends HttpServlet {
+public class DBCheckpointServlet extends HttpServlet
+ implements BootstrapStateHandler {
private static final String FIELD_NAME =
OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST + "[]";
@@ -69,6 +71,7 @@ public class DBCheckpointServlet extends HttpServlet {
private boolean aclEnabled;
private boolean isSpnegoEnabled;
private transient OzoneAdmins admins;
+ private transient BootstrapStateHandler.Lock lock;
public void initialize(DBStore store, DBCheckpointMetrics metrics,
boolean omAclEnabled,
@@ -87,6 +90,7 @@ public class DBCheckpointServlet extends HttpServlet {
this.aclEnabled = omAclEnabled;
this.admins = new OzoneAdmins(allowedAdminUsers, allowedAdminGroups);
this.isSpnegoEnabled = isSpnegoAuthEnabled;
+ lock = new Lock();
}
private boolean hasPermission(UserGroupInformation user) {
@@ -145,28 +149,29 @@ public class DBCheckpointServlet extends HttpServlet {
}
DBCheckpoint checkpoint = null;
- try {
- boolean flush = false;
- String flushParam =
- request.getParameter(OZONE_DB_CHECKPOINT_REQUEST_FLUSH);
- if (StringUtils.isNotEmpty(flushParam)) {
- flush = Boolean.parseBoolean(flushParam);
- }
- List<String> receivedSstList = new ArrayList<>();
- List<String> excludedSstList = new ArrayList<>();
- String[] sstParam = isFormData ?
- parseFormDataParameters(request) : request.getParameterValues(
- OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST);
- if (sstParam != null) {
- receivedSstList.addAll(
- Arrays.stream(sstParam)
- .filter(s -> s.endsWith(ROCKSDB_SST_SUFFIX))
- .distinct()
- .collect(Collectors.toList()));
- LOG.info("Received excluding SST {}", receivedSstList);
- }
+ boolean flush = false;
+ String flushParam =
+ request.getParameter(OZONE_DB_CHECKPOINT_REQUEST_FLUSH);
+ if (StringUtils.isNotEmpty(flushParam)) {
+ flush = Boolean.parseBoolean(flushParam);
+ }
+ List<String> receivedSstList = new ArrayList<>();
+ List<String> excludedSstList = new ArrayList<>();
+ String[] sstParam = isFormData ?
+ parseFormDataParameters(request) : request.getParameterValues(
+ OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST);
+ if (sstParam != null) {
+ receivedSstList.addAll(
+ Arrays.stream(sstParam)
+ .filter(s -> s.endsWith(ROCKSDB_SST_SUFFIX))
+ .distinct()
+ .collect(Collectors.toList()));
+ LOG.info("Received excluding SST {}", receivedSstList);
+ }
+
+ try (BootstrapStateHandler.Lock lock = getBootstrapStateLock().lock()) {
checkpoint = dbStore.getCheckpoint(flush);
if (checkpoint == null || checkpoint.getCheckpointLocation() == null) {
LOG.error("Unable to process metadata snapshot request. " +
@@ -301,4 +306,27 @@ public class DBCheckpointServlet extends HttpServlet {
writeDBCheckpointToStream(checkpoint, destination,
toExcludeList, excludedList);
}
+
+ @Override
+ public BootstrapStateHandler.Lock getBootstrapStateLock() {
+ return lock;
+ }
+
+ /**
+ * This lock is a no-op but can overridden by child classes.
+ */
+ public static class Lock extends BootstrapStateHandler.Lock {
+ public Lock() {
+ }
+
+ @Override
+ public BootstrapStateHandler.Lock lock()
+ throws InterruptedException {
+ return this;
+ }
+
+ @Override
+ public void unlock() {
+ }
+ }
}
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java
index 4217d347d1..bb51702386 100644
---
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMMetrics;
import org.apache.hadoop.hdds.scm.server.SCMDBCheckpointServlet;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.apache.hadoop.hdds.utils.DBCheckpointServlet;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConsts;
@@ -187,6 +188,8 @@ public class TestSCMDbCheckpointServlet {
}
});
+ when(scmDbCheckpointServletMock.getBootstrapStateLock()).thenReturn(
+ new DBCheckpointServlet.Lock());
scmDbCheckpointServletMock.init();
long initialCheckpointCount =
scmMetrics.getDBCheckpointMetrics().getNumCheckpoints();
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
index 2a6b35ae63..c16b31cc36 100644
---
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
@@ -49,6 +49,7 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -57,6 +58,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
+import org.apache.hadoop.hdds.utils.db.DBStore;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.TestDataUtil;
@@ -128,7 +130,6 @@ public class TestOMDbCheckpointServlet {
private HttpServletRequest requestMock = null;
private HttpServletResponse responseMock = null;
private OMDBCheckpointServlet omDbCheckpointServletMock = null;
- private BootstrapStateHandler.Lock lock;
private File metaDir;
private String snapshotDirName;
private String snapshotDirName2;
@@ -137,7 +138,7 @@ public class TestOMDbCheckpointServlet {
private String method;
private File folder;
private static final String FABRICATED_FILE_NAME = "fabricatedFile.sst";
-
+ private FileOutputStream fileOutputStream;
/**
* Create a MiniDFSCluster for testing.
* <p>
@@ -153,7 +154,7 @@ public class TestOMDbCheckpointServlet {
tempFile = File.createTempFile("temp_" + System
.currentTimeMillis(), ".tar");
- FileOutputStream fileOutputStream = new FileOutputStream(tempFile);
+ fileOutputStream = new FileOutputStream(tempFile);
servletOutputStream = new ServletOutputStream() {
@Override
@@ -193,7 +194,8 @@ public class TestOMDbCheckpointServlet {
omDbCheckpointServletMock =
mock(OMDBCheckpointServlet.class);
- lock = new OMDBCheckpointServlet.Lock(cluster.getOzoneManager());
+ BootstrapStateHandler.Lock lock =
+ new OMDBCheckpointServlet.Lock(cluster.getOzoneManager());
doCallRealMethod().when(omDbCheckpointServletMock).init();
requestMock = mock(HttpServletRequest.class);
@@ -370,11 +372,31 @@ public class TestOMDbCheckpointServlet {
when(requestMock.getParameter(OZONE_DB_CHECKPOINT_INCLUDE_SNAPSHOT_DATA))
.thenReturn("true");
+ // Create a "spy" dbstore keep track of the checkpoint.
+ OzoneManager om = cluster.getOzoneManager();
+ DBStore dbStore = om.getMetadataManager().getStore();
+ DBStore spyDbStore = spy(dbStore);
+ AtomicReference<DBCheckpoint> realCheckpoint = new AtomicReference<>();
+ when(spyDbStore.getCheckpoint(true)).thenAnswer(b -> {
+ DBCheckpoint checkpoint = spy(dbStore.getCheckpoint(true));
+ // Don't delete the checkpoint, because we need to compare it
+ // with the snapshot data.
+ doNothing().when(checkpoint).cleanupCheckpoint();
+ realCheckpoint.set(checkpoint);
+ return checkpoint;
+ });
+
+ // Init the mock with the spyDbstore
+ doCallRealMethod().when(omDbCheckpointServletMock).initialize(
+ any(), any(), eq(false), any(), any(), eq(false));
+ omDbCheckpointServletMock.initialize(
+ spyDbStore, om.getMetrics().getDBCheckpointMetrics(),
+ false, om.getOmAdminUsernames(), om.getOmAdminGroups(), false);
+
// Get the tarball.
- try (FileOutputStream fileOutputStream = new FileOutputStream(tempFile)) {
- omDbCheckpointServletMock.writeDbDataToStream(dbCheckpoint, requestMock,
- fileOutputStream, new ArrayList<>(), new ArrayList<>());
- }
+ when(responseMock.getOutputStream()).thenReturn(servletOutputStream);
+ omDbCheckpointServletMock.doGet(requestMock, responseMock);
+ dbCheckpoint = realCheckpoint.get();
// Untar the file into a temp folder to be examined.
String testDirName = folder.getAbsolutePath();
@@ -805,7 +827,7 @@ public class TestOMDbCheckpointServlet {
// Confirm that servlet takes the lock when none of the other
// handlers have it.
Future<Boolean> servletTest = checkLock(spyServlet, executorService);
- Assert.assertTrue(servletTest.get(10000, TimeUnit.MILLISECONDS));
+ Assertions.assertTrue(servletTest.get(10000, TimeUnit.MILLISECONDS));
executorService.shutdownNow();
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java
index 4db64fba41..f6e14fe97a 100644
---
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java
@@ -78,8 +78,7 @@ import static
org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.truncateFileNa
* If Kerberos is not enabled, simply append the login username to
* `ozone.administrator`, e.g. `scm`
*/
-public class OMDBCheckpointServlet extends DBCheckpointServlet
- implements BootstrapStateHandler {
+public class OMDBCheckpointServlet extends DBCheckpointServlet {
private static final Logger LOG =
LoggerFactory.getLogger(OMDBCheckpointServlet.class);
@@ -135,8 +134,7 @@ public class OMDBCheckpointServlet extends
DBCheckpointServlet
// Map of link to path.
Map<Path, Path> hardLinkFiles = new HashMap<>();
- try (BootstrapStateHandler.Lock lock = getBootstrapStateLock().lock();
- TarArchiveOutputStream archiveOutputStream =
+ try (TarArchiveOutputStream archiveOutputStream =
new TarArchiveOutputStream(destination)) {
archiveOutputStream
.setLongFileMode(TarArchiveOutputStream.LONGFILE_POSIX);
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]