Skip to content

Commit

Permalink
HDFS-17532. RBF: Allow router state store cache update to overwrite a…
Browse files Browse the repository at this point in the history
…nd delete in parallel (#6839)
  • Loading branch information
Felix Nguyen authored May 28, 2024
1 parent 1baf0e8 commit 74d30a5
Show file tree
Hide file tree
Showing 9 changed files with 113 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -218,6 +218,9 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic {
FEDERATION_STORE_PREFIX + "driver.class";
public static final Class<? extends StateStoreDriver>
FEDERATION_STORE_DRIVER_CLASS_DEFAULT = StateStoreZooKeeperImpl.class;
public static final String FEDERATION_STORE_DRIVER_ASYNC_OVERRIDE_MAX_THREADS =
FEDERATION_STORE_PREFIX + "driver.async.override.max.threads";
public static final int FEDERATION_STORE_DRIVER_ASYNC_OVERRIDE_MAX_THREADS_DEFAULT = -1;

public static final String FEDERATION_STORE_CONNECTION_TEST_MS =
FEDERATION_STORE_PREFIX + "connection.test";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
Expand Down Expand Up @@ -173,7 +172,7 @@ private boolean isUpdateTime() {
*/
public void overrideExpiredRecords(QueryResult<R> query) throws IOException {
List<R> commitRecords = new ArrayList<>();
List<R> toDeleteRecords = new ArrayList<>();
List<R> deleteRecords = new ArrayList<>();
List<R> newRecords = query.getRecords();
long currentDriverTime = query.getTimestamp();
if (newRecords == null || currentDriverTime <= 0) {
Expand All @@ -184,22 +183,18 @@ public void overrideExpiredRecords(QueryResult<R> query) throws IOException {
if (record.shouldBeDeleted(currentDriverTime)) {
String recordName = StateStoreUtils.getRecordName(record.getClass());
LOG.info("State Store record to delete {}: {}", recordName, record);
toDeleteRecords.add(record);
deleteRecords.add(record);
} else if (!record.isExpired() && record.checkExpired(currentDriverTime)) {
String recordName = StateStoreUtils.getRecordName(record.getClass());
LOG.info("Override State Store record {}: {}", recordName, record);
commitRecords.add(record);
}
}
if (commitRecords.size() > 0) {
getDriver().putAll(commitRecords, true, false);
}
if (!toDeleteRecords.isEmpty()) {
for (Map.Entry<R, Boolean> entry : getDriver().removeMultiple(toDeleteRecords).entrySet()) {
if (entry.getValue()) {
newRecords.remove(entry.getKey());
}
}
List<R> removedRecords = getDriver().handleOverwriteAndDelete(commitRecords, deleteRecords);
// In driver async mode, driver will return null and skip the next block.
// newRecords might be stale as a result but will sort itself out the next override cycle.
if (removedRecords != null && !removedRecords.isEmpty()) {
newRecords.removeAll(removedRecords);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,22 @@
*/
package org.apache.hadoop.hdfs.server.federation.store.driver;

import java.io.IOException;
import java.net.InetAddress;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.federation.metrics.StateStoreMetrics;
import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
Expand Down Expand Up @@ -54,6 +63,9 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
/** State Store metrics. */
private StateStoreMetrics metrics;

/** Thread pool to delegate overwrite and deletion asynchronously. */
private ThreadPoolExecutor executor = null;

/**
* Initialize the state store connection.
*
Expand Down Expand Up @@ -88,6 +100,18 @@ public boolean init(final Configuration config, final String id,
return false;
}
}

int nThreads = conf.getInt(
RBFConfigKeys.FEDERATION_STORE_DRIVER_ASYNC_OVERRIDE_MAX_THREADS,
RBFConfigKeys.FEDERATION_STORE_DRIVER_ASYNC_OVERRIDE_MAX_THREADS_DEFAULT);
if (nThreads > 0) {
executor = new ThreadPoolExecutor(nThreads, nThreads, 1L, TimeUnit.MINUTES,
new LinkedBlockingQueue<>());
executor.allowCoreThreadTimeOut(true);
LOG.info("Init StateStoreDriver in async mode with {} threads.", nThreads);
} else {
LOG.info("Init StateStoreDriver in sync mode.");
}
return true;
}

Expand Down Expand Up @@ -169,7 +193,12 @@ public void verifyDriverReady() throws StateStoreUnavailableException {
*
* @throws Exception if something goes wrong while closing the state store driver connection.
*/
public abstract void close() throws Exception;
public void close() throws Exception {
if (executor != null) {
executor.shutdown();
executor = null;
}
}

/**
* Returns the current time synchronization from the underlying store.
Expand Down Expand Up @@ -206,4 +235,62 @@ private String getHostname() {
}
return hostname;
}

/**
* Try to overwrite records in commitRecords and remove records in deleteRecords.
* Should return null if async mode is used. Else return removed records.
* @param commitRecords records to overwrite in state store
* @param deleteRecords records to remove from state store
* @param <R> record class
* @throws IOException when there is a failure during overwriting or deletion
* @return null if async mode is used, else removed records
*/
public <R extends BaseRecord> List<R> handleOverwriteAndDelete(List<R> commitRecords,
List<R> deleteRecords) throws IOException {
List<R> result = null;
try {
// Overwrite all expired records.
if (commitRecords != null && !commitRecords.isEmpty()) {
Runnable overwriteCallable =
() -> {
try {
putAll(commitRecords, true, false);
} catch (IOException e) {
throw new RuntimeException(e);
}
};
if (executor != null) {
executor.execute(overwriteCallable);
} else {
overwriteCallable.run();
}
}

// Delete all deletable records.
if (deleteRecords != null && !deleteRecords.isEmpty()) {
Map<R, Boolean> removedRecords = new HashMap<>();
Runnable deletionCallable = () -> {
try {
removedRecords.putAll(removeMultiple(deleteRecords));
} catch (IOException e) {
throw new RuntimeException(e);
}
};
if (executor != null) {
executor.execute(deletionCallable);
} else {
result = new ArrayList<>();
deletionCallable.run();
for (Map.Entry<R, Boolean> entry : removedRecords.entrySet()) {
if (entry.getValue()) {
result.add(entry.getKey());
}
}
}
}
} catch (Exception e) {
throw new IOException(e);
}
return result;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,7 @@ public boolean initDriver() {

@Override
public void close() throws Exception {
super.close();
if (this.concurrentStoreAccessPool != null) {
this.concurrentStoreAccessPool.shutdown();
boolean isTerminated = this.concurrentStoreAccessPool.awaitTermination(5, TimeUnit.SECONDS);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,7 @@ public boolean isDriverReady() {

@Override
public void close() throws Exception {
super.close();
connectionFactory.shutdown();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,7 @@ public void setEnableConcurrent(boolean enableConcurrent) {

@Override
public void close() throws Exception {
super.close();
if (executorService != null) {
executorService.shutdown();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -370,6 +370,16 @@
</description>
</property>

<property>
<name>dfs.federation.router.store.driver.async.override.max.threads</name>
<value>-1</value>
<description>
Number of threads used by StateStoreDriver to overwrite and delete records asynchronously.
Only used by MembershipStore and RouterStore. Non-positive values will make StateStoreDriver
run in sync mode.
</description>
</property>

<property>
<name>dfs.federation.router.store.connection.test</name>
<value>60000</value>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -469,6 +469,7 @@ The connection to the State Store and the internal caching at the Router.
| dfs.federation.router.store.connection.test | 60000 | How often to check for the connection to the State Store in milliseconds. |
| dfs.federation.router.cache.ttl | 60000 | How often to refresh the State Store caches in milliseconds. |
| dfs.federation.router.store.membership.expiration | 300000 | Expiration time in milliseconds for a membership record. |
| dfs.federation.router.store.driver.async.override.max.threads | | Number of threads to overwrite and delete records asynchronously when overriding. |
| dfs.federation.router.mount-table.cache.update | false | If true, Mount table cache is updated whenever a mount table entry is added, modified or removed for all the routers. |
| dfs.federation.router.mount-table.cache.update.timeout | 1m | Max time to wait for all the routers to finish their mount table cache update. |
| dfs.federation.router.mount-table.cache.update.client.max.time | 5m | Max time a RouterClient connection can be cached. |
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ public boolean isDriverReady() {

@Override
public void close() throws Exception {
super.close();
VALUE_MAP.clear();
initialized = false;
}
Expand Down

0 comments on commit 74d30a5

Please sign in to comment.