You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2023/01/19 17:31:17 UTC

[GitHub] [ozone] sadanand48 commented on a diff in pull request #4190: HDDS-7800. Use persistent map to generate snapshot diff report

sadanand48 commented on code in PR #4190:
URL: https://github.com/apache/ozone/pull/4190#discussion_r1081599774


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -271,61 +359,130 @@ private Set<String> getDeltaFiles(OmSnapshot fromSnapshot,
   }
 
   private List<DiffReportEntry> generateDiffReport(
-      final Set<Long> objectIDsToCheck,
-      final Map<Long, String> oldObjIdToKeyMap,
-      final Map<Long, String> newObjIdToKeyMap) {
-
-    final List<DiffReportEntry> deleteDiffs = new ArrayList<>();
-    final List<DiffReportEntry> renameDiffs = new ArrayList<>();
-    final List<DiffReportEntry> createDiffs = new ArrayList<>();
-    final List<DiffReportEntry> modifyDiffs = new ArrayList<>();
-
-
-    for (Long id : objectIDsToCheck) {
-      /*
-       * This key can be
-       * -> Created after the old snapshot was taken, which means it will be
-       *    missing in oldKeyTable and present in newKeyTable.
-       * -> Deleted after the old snapshot was taken, which means it will be
-       *    present in oldKeyTable and missing in newKeyTable.
-       * -> Modified after the old snapshot was taken, which means it will be
-       *    present in oldKeyTable and present in newKeyTable with same
-       *    Object ID but with different metadata.
-       * -> Renamed after the old snapshot was taken, which means it will be
-       *    present in oldKeyTable and present in newKeyTable but with different
-       *    name and same Object ID.
-       */
-
-      final String oldKeyName = oldObjIdToKeyMap.get(id);
-      final String newKeyName = newObjIdToKeyMap.get(id);
-
-      if (oldKeyName == null && newKeyName == null) {
-        // This cannot happen.
-        continue;
+      final String requestId,
+      final PersistentMap<Long, byte[]> objectIDsToCheck,
+      final PersistentMap<Long, String> oldObjIdToKeyMap,
+      final PersistentMap<Long, String> newObjIdToKeyMap
+  ) throws RocksDBException, IOException {
+
+    // RequestId is prepended to column family name to make it unique
+    // for request.
+    ColumnFamilyHandle deleteDiffColumnFamily = null;
+    ColumnFamilyHandle renameDiffColumnFamily = null;
+    ColumnFamilyHandle createDiffColumnFamily = null;
+    ColumnFamilyHandle modifyDiffColumnFamily = null;
+
+    try {
+      deleteDiffColumnFamily = rocksDB.createColumnFamily(
+          new ColumnFamilyDescriptor(
+              codecRegistry.asRawData(requestId + "-deleteDiff"),
+              new ColumnFamilyOptions()));
+      renameDiffColumnFamily = rocksDB.createColumnFamily(
+          new ColumnFamilyDescriptor(
+              codecRegistry.asRawData(requestId + "-renameDiff"),
+              new ColumnFamilyOptions()));
+      createDiffColumnFamily = rocksDB.createColumnFamily(
+          new ColumnFamilyDescriptor(
+              codecRegistry.asRawData(requestId + "-createDiff"),
+              new ColumnFamilyOptions()));
+      modifyDiffColumnFamily = rocksDB.createColumnFamily(
+          new ColumnFamilyDescriptor(
+              codecRegistry.asRawData(requestId + "-modifyDiff"),
+              new ColumnFamilyOptions()));
+
+      final PersistentMap<Long, DiffReportEntry> deleteDiffs =
+          createDiffReportPersistentMap(deleteDiffColumnFamily);
+      final PersistentMap<Long, DiffReportEntry> renameDiffs =
+          createDiffReportPersistentMap(renameDiffColumnFamily);
+      final PersistentMap<Long, DiffReportEntry> createDiffs =
+          createDiffReportPersistentMap(createDiffColumnFamily);
+      final PersistentMap<Long, DiffReportEntry> modifyDiffs =
+          createDiffReportPersistentMap(modifyDiffColumnFamily);
+
+
+      long deleteCounter = 0L;
+      long renameCounter = 0L;
+      long createCounter = 0L;
+      long modifyCounter = 0L;
+
+      Iterator<Long> objectIdsIterator = objectIDsToCheck.getKeyIterator();
+      while (objectIdsIterator.hasNext()) {
+        Long id = objectIdsIterator.next();
+        /*
+         * This key can be
+         * -> Created after the old snapshot was taken, which means it will be
+         *    missing in oldKeyTable and present in newKeyTable.
+         * -> Deleted after the old snapshot was taken, which means it will be
+         *    present in oldKeyTable and missing in newKeyTable.
+         * -> Modified after the old snapshot was taken, which means it will be
+         *    present in oldKeyTable and present in newKeyTable with same
+         *    Object ID but with different metadata.
+         * -> Renamed after the old snapshot was taken, which means it will be
+         *    present in oldKeyTable and present in newKeyTable but with
+         *    different name and same Object ID.
+         */
+
+        final String oldKeyName = oldObjIdToKeyMap.get(id);
+        final String newKeyName = newObjIdToKeyMap.get(id);
+
+        if (oldKeyName == null && newKeyName == null) {
+          // This cannot happen.
+          throw new IllegalStateException("Old and new key name both are null");
+        } else if (oldKeyName == null) { // Key Created.
+          createDiffs.put(createCounter++,
+              DiffReportEntry.of(DiffType.CREATE, newKeyName));
+        } else if (newKeyName == null) { // Key Deleted.
+          deleteDiffs.put(deleteCounter++,
+              DiffReportEntry.of(DiffType.DELETE, oldKeyName));
+        } else if (oldKeyName.equals(newKeyName)) { // Key modified.
+          modifyDiffs.put(modifyCounter++,
+              DiffReportEntry.of(DiffType.MODIFY, newKeyName));
+        } else { // Key Renamed.
+          renameDiffs.put(renameCounter++,
+              DiffReportEntry.of(DiffType.RENAME, oldKeyName, newKeyName));
+        }
       }
 
-      // Key Created.
-      if (oldKeyName == null) {
-        createDiffs.add(DiffReportEntry.of(DiffType.CREATE, newKeyName));
-        continue;
+      return aggregateDiffReports(deleteDiffs,
+          renameDiffs,
+          createDiffs,
+          modifyDiffs);
+    } finally {
+      if (deleteDiffColumnFamily != null) {
+        rocksDB.dropColumnFamily(deleteDiffColumnFamily);
+        deleteDiffColumnFamily.close();
       }
-
-      // Key Deleted.
-      if (newKeyName == null) {
-        deleteDiffs.add(DiffReportEntry.of(DiffType.DELETE, oldKeyName));
-        continue;
+      if (renameDiffColumnFamily != null) {

Review Comment:
   One benefit I see of storing these diffs is if we don't drop them here and persist them with the from and to snapshot Info we can easily use it to store precomputed snapdiffs.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -271,61 +359,130 @@ private Set<String> getDeltaFiles(OmSnapshot fromSnapshot,
   }
 
   private List<DiffReportEntry> generateDiffReport(
-      final Set<Long> objectIDsToCheck,
-      final Map<Long, String> oldObjIdToKeyMap,
-      final Map<Long, String> newObjIdToKeyMap) {
-
-    final List<DiffReportEntry> deleteDiffs = new ArrayList<>();
-    final List<DiffReportEntry> renameDiffs = new ArrayList<>();
-    final List<DiffReportEntry> createDiffs = new ArrayList<>();
-    final List<DiffReportEntry> modifyDiffs = new ArrayList<>();
-
-
-    for (Long id : objectIDsToCheck) {
-      /*
-       * This key can be
-       * -> Created after the old snapshot was taken, which means it will be
-       *    missing in oldKeyTable and present in newKeyTable.
-       * -> Deleted after the old snapshot was taken, which means it will be
-       *    present in oldKeyTable and missing in newKeyTable.
-       * -> Modified after the old snapshot was taken, which means it will be
-       *    present in oldKeyTable and present in newKeyTable with same
-       *    Object ID but with different metadata.
-       * -> Renamed after the old snapshot was taken, which means it will be
-       *    present in oldKeyTable and present in newKeyTable but with different
-       *    name and same Object ID.
-       */
-
-      final String oldKeyName = oldObjIdToKeyMap.get(id);
-      final String newKeyName = newObjIdToKeyMap.get(id);
-
-      if (oldKeyName == null && newKeyName == null) {
-        // This cannot happen.
-        continue;
+      final String requestId,
+      final PersistentMap<Long, byte[]> objectIDsToCheck,
+      final PersistentMap<Long, String> oldObjIdToKeyMap,
+      final PersistentMap<Long, String> newObjIdToKeyMap
+  ) throws RocksDBException, IOException {
+
+    // RequestId is prepended to column family name to make it unique
+    // for request.
+    ColumnFamilyHandle deleteDiffColumnFamily = null;
+    ColumnFamilyHandle renameDiffColumnFamily = null;
+    ColumnFamilyHandle createDiffColumnFamily = null;
+    ColumnFamilyHandle modifyDiffColumnFamily = null;
+
+    try {
+      deleteDiffColumnFamily = rocksDB.createColumnFamily(
+          new ColumnFamilyDescriptor(
+              codecRegistry.asRawData(requestId + "-deleteDiff"),
+              new ColumnFamilyOptions()));
+      renameDiffColumnFamily = rocksDB.createColumnFamily(
+          new ColumnFamilyDescriptor(
+              codecRegistry.asRawData(requestId + "-renameDiff"),
+              new ColumnFamilyOptions()));
+      createDiffColumnFamily = rocksDB.createColumnFamily(
+          new ColumnFamilyDescriptor(
+              codecRegistry.asRawData(requestId + "-createDiff"),
+              new ColumnFamilyOptions()));
+      modifyDiffColumnFamily = rocksDB.createColumnFamily(
+          new ColumnFamilyDescriptor(
+              codecRegistry.asRawData(requestId + "-modifyDiff"),
+              new ColumnFamilyOptions()));
+
+      final PersistentMap<Long, DiffReportEntry> deleteDiffs =
+          createDiffReportPersistentMap(deleteDiffColumnFamily);

Review Comment:
   Thanks @hemantk-12 for working on this. I had some questions
   1. The jira states that the objects would be persisted only if they are too large to fit in memory but I see here we are always persisting these objects, are we planning to always persist?
   2. For the diffs i.e create , delete etc I see we are persisting in Rocksdb and later populating all the entries inside a List so I don't think we are gaining much by persisting because ultimately we are populating it in memory ( in `aggregateDiffReports`).  
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org