otterc commented on code in PR #35906:
URL: https://github.com/apache/spark/pull/35906#discussion_r898325741


##########
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java:
##########
@@ -656,6 +771,206 @@ public void registerExecutor(String appId, 
ExecutorShuffleInfo executorInfo) {
     }
   }
 
+  /**
+   * Close the DB during shutdown
+   */
+  @Override
+  public void close() {
+    if (db != null) {
+      try {
+        db.close();
+      } catch (IOException e) {
+        logger.error("Exception closing leveldb with registered app paths info 
and "
+            + "shuffle partition info", e);
+      }
+    }
+  }
+
+  /**
+   * Write the application attempt's local path information to the DB
+   */
+  private void writeAppPathsInfoToDb(String appId, int attemptId, AppPathsInfo 
appPathsInfo) {
+    if (db != null) {
+      try {
+        byte[] key = getDbAppAttemptPathsKey(new AppAttemptId(appId, 
attemptId));
+        String valueStr = mapper.writeValueAsString(appPathsInfo);
+        byte[] value = valueStr.getBytes(StandardCharsets.UTF_8);
+        db.put(key, value);
+      } catch (Exception e) {
+        logger.error("Error saving registered app paths info", e);
+      }
+    }
+  }
+
+  /**
+   * Write the finalized shuffle merge partition information into the DB
+   */
+  private void writeAppAttemptShuffleMergeInfoToDB(
+      AppAttemptShuffleMergeId appAttemptShuffleMergeId) {
+    if (db != null) {
+      // Write AppAttemptShuffleMergeId into LevelDB for finalized shuffles
+      try{
+        byte[] dbKey = 
getDbAppAttemptShufflePartitionKey(appAttemptShuffleMergeId);
+        db.put(dbKey, new byte[0]);
+      } catch (Exception e) {
+        logger.error("Error saving active app shuffle partition", e);
+      }
+    }
+
+  }
+
+  /**
+   * Parse the DB key with the prefix and the expected return value type
+   */
+  private <T> T parseDbKey(String key, String prefix, Class<T> valueType) 
throws IOException {
+    String json = key.substring(prefix.length() + 1);
+    return mapper.readValue(json, valueType);
+  }
+
+  /**
+   * Generate AppAttemptId from the DB key
+   */
+  private AppAttemptId parseDbAppAttemptPathsKey(String key) throws 
IOException {
+    return parseDbKey(key, APP_ATTEMPT_PATH_KEY_PREFIX, AppAttemptId.class);
+  }
+
+  /**
+   * Generate AppAttemptShuffleMergeId from the DB key
+   */
+  private AppAttemptShuffleMergeId parseDbAppAttemptShufflePartitionKey(
+      String key) throws IOException {
+    return parseDbKey(
+        key, APP_ATTEMPT_SHUFFLE_FINALIZE_STATUS_KEY_PREFIX, 
AppAttemptShuffleMergeId.class);
+  }
+
+  /**
+   * Generate the DB key with the key object and the specified string prefix
+   */
+  private byte[] getDbKey(Object key, String prefix) {
+    // We add a common prefix on all the keys so we can find them in the DB
+    try {
+      String keyJsonString = prefix + DB_KEY_DELIMITER + 
mapper.writeValueAsString(key);
+      return keyJsonString.getBytes(StandardCharsets.UTF_8);
+    } catch (Exception exception) {
+      logger.error("Exception while generating the DB key {}", key);
+      return null;
+    }
+  }
+
+  /**
+   * Generate the DB key from AppAttemptShuffleMergeId object
+   */
+  private byte[] getDbAppAttemptShufflePartitionKey(
+      AppAttemptShuffleMergeId appAttemptShuffleMergeId) {
+    return getDbKey(appAttemptShuffleMergeId, 
APP_ATTEMPT_SHUFFLE_FINALIZE_STATUS_KEY_PREFIX);
+  }
+
+  /**
+   * Generate the DB key from AppAttemptId object
+   */
+  private byte[] getDbAppAttemptPathsKey(AppAttemptId appAttemptId){
+    return getDbKey(appAttemptId, APP_ATTEMPT_PATH_KEY_PREFIX);
+  }
+
+  /**
+   * Reload the DB to recover the meta data stored in the hashmap for merged 
shuffles.
+   * The application attempts local paths information will be firstly 
reloaded, and then
+   * the finalized shuffle merges will be updated.
+   */
+  @VisibleForTesting
+  void reloadAndCleanUpAppShuffleInfo(DB db) throws IOException {
+    logger.info("Reload applications merged shuffle information from DB");
+    reloadActiveAppAttemptsPathInfo(db);
+    reloadFinalizedAppAttemptsShuffleMergeInfo(db);
+  }
+
+  /**
+   * Reload application attempts local paths information.
+   */
+  private void reloadActiveAppAttemptsPathInfo(DB db) throws IOException {
+    if (db != null) {
+      DBIterator itr = db.iterator();
+      itr.seek(APP_ATTEMPT_PATH_KEY_PREFIX.getBytes(StandardCharsets.UTF_8));
+      while (itr.hasNext()) {
+        Map.Entry<byte[], byte[]> e = itr.next();
+        String key = new String(e.getKey(), StandardCharsets.UTF_8);
+        if (!key.startsWith(APP_ATTEMPT_PATH_KEY_PREFIX)) {
+          break;
+        }
+        AppAttemptId appAttemptId = parseDbAppAttemptPathsKey(key);
+        AppPathsInfo appPathsInfo = mapper.readValue(e.getValue(), 
AppPathsInfo.class);
+        appsShuffleInfo.put(appAttemptId.appId, new AppShuffleInfo(
+            appAttemptId.appId, appAttemptId.attemptId, appPathsInfo));
+      }
+    }
+  }
+
+  /**
+   * Reload the finalized shuffle merges.
+   * Since the deletion of finalized shuffle merges are triggered 
asynchronously, there can be
+   * cases that deletions miss the execution during restart. Those dangling 
finalized shuffle merge
+   * keys in the DB will be cleaned up during this reload, if there are no 
relevant application
+   * attempts local paths information registered in the hashmap.
+   */
+  private void reloadFinalizedAppAttemptsShuffleMergeInfo(DB db) throws 
IOException {
+    List<byte[]> dbKeysToBeRemoved = new ArrayList<>();
+    if (db != null) {
+      DBIterator itr = db.iterator();
+      
itr.seek(APP_ATTEMPT_SHUFFLE_FINALIZE_STATUS_KEY_PREFIX.getBytes(StandardCharsets.UTF_8));
+      while (itr.hasNext()) {
+        Map.Entry<byte[], byte[]> e = itr.next();
+        String key = new String(e.getKey(), StandardCharsets.UTF_8);
+        if (!key.startsWith(APP_ATTEMPT_SHUFFLE_FINALIZE_STATUS_KEY_PREFIX)) {
+          break;
+        }
+        AppAttemptShuffleMergeId partitionId = 
parseDbAppAttemptShufflePartitionKey(key);

Review Comment:
   `parseDbAppAttemptShufflePartitionKey` will return null, if there was an 
exception because one of the other methods it calls was catching the Exception 
and returns `null`. We need to make that other method not return null and throw 
exception so the callers can handle it.



-- 
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: reviews-unsubscr...@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to