|
@@ -25,14 +25,24 @@ import java.io.BufferedReader;
|
|
|
import java.io.BufferedWriter;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Map.Entry;
|
|
|
+import java.util.concurrent.Callable;
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
+import java.util.concurrent.ExecutorService;
|
|
|
+import java.util.concurrent.Future;
|
|
|
+import java.util.concurrent.LinkedBlockingQueue;
|
|
|
+import java.util.concurrent.ThreadPoolExecutor;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
import java.util.regex.Matcher;
|
|
|
import java.util.regex.Pattern;
|
|
|
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
+
|
|
|
import org.apache.hadoop.hdfs.server.federation.metrics.StateStoreMetrics;
|
|
|
import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
|
|
|
import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
|
|
@@ -69,6 +79,8 @@ public abstract class StateStoreFileBaseImpl
|
|
|
/** If it is initialized. */
|
|
|
private boolean initialized = false;
|
|
|
|
|
|
+ private ExecutorService concurrentStoreAccessPool;
|
|
|
+
|
|
|
|
|
|
/**
|
|
|
* Get the reader of a record for the file system.
|
|
@@ -137,6 +149,8 @@ public abstract class StateStoreFileBaseImpl
|
|
|
*/
|
|
|
protected abstract String getRootDir();
|
|
|
|
|
|
+ protected abstract int getConcurrentFilesAccessNumThreads();
|
|
|
+
|
|
|
/**
|
|
|
* Set the driver as initialized.
|
|
|
*
|
|
@@ -168,9 +182,31 @@ public abstract class StateStoreFileBaseImpl
|
|
|
return false;
|
|
|
}
|
|
|
setInitialized(true);
|
|
|
+ int threads = getConcurrentFilesAccessNumThreads();
|
|
|
+ if (threads > 1) {
|
|
|
+ this.concurrentStoreAccessPool =
|
|
|
+ new ThreadPoolExecutor(threads, threads, 0L, TimeUnit.MILLISECONDS,
|
|
|
+ new LinkedBlockingQueue<>(),
|
|
|
+ new ThreadFactoryBuilder()
|
|
|
+ .setNameFormat("state-store-file-based-concurrent-%d")
|
|
|
+ .setDaemon(true).build());
|
|
|
+ LOG.info("File based state store will be accessed concurrently with {} max threads", threads);
|
|
|
+ } else {
|
|
|
+ LOG.info("File based state store will be accessed serially");
|
|
|
+ }
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public void close() throws Exception {
|
|
|
+ if (this.concurrentStoreAccessPool != null) {
|
|
|
+ this.concurrentStoreAccessPool.shutdown();
|
|
|
+ boolean isTerminated = this.concurrentStoreAccessPool.awaitTermination(5, TimeUnit.SECONDS);
|
|
|
+ LOG.info("Concurrent store access pool is terminated: {}", isTerminated);
|
|
|
+ this.concurrentStoreAccessPool = null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public <T extends BaseRecord> boolean initRecordStorage(
|
|
|
String className, Class<T> recordClass) {
|
|
@@ -198,22 +234,29 @@ public abstract class StateStoreFileBaseImpl
|
|
|
verifyDriverReady();
|
|
|
long start = monotonicNow();
|
|
|
StateStoreMetrics metrics = getMetrics();
|
|
|
- List<T> ret = new ArrayList<>();
|
|
|
+ List<T> result = Collections.synchronizedList(new ArrayList<>());
|
|
|
try {
|
|
|
String path = getPathForClass(clazz);
|
|
|
List<String> children = getChildren(path);
|
|
|
- for (String child : children) {
|
|
|
- String pathRecord = path + "/" + child;
|
|
|
- if (child.endsWith(TMP_MARK)) {
|
|
|
- LOG.debug("There is a temporary file {} in {}", child, path);
|
|
|
- if (isOldTempRecord(child)) {
|
|
|
- LOG.warn("Removing {} as it's an old temporary record", child);
|
|
|
- remove(pathRecord);
|
|
|
- }
|
|
|
- } else {
|
|
|
- T record = getRecord(pathRecord, clazz);
|
|
|
- ret.add(record);
|
|
|
+ List<Callable<Void>> callables = new ArrayList<>();
|
|
|
+ children.forEach(child -> callables.add(
|
|
|
+ () -> getRecordsFromFileAndRemoveOldTmpRecords(clazz, result, path, child)));
|
|
|
+ if (this.concurrentStoreAccessPool != null) {
|
|
|
+ // Read records concurrently
|
|
|
+ List<Future<Void>> futures = this.concurrentStoreAccessPool.invokeAll(callables);
|
|
|
+ for (Future<Void> future : futures) {
|
|
|
+ future.get();
|
|
|
}
|
|
|
+ } else {
|
|
|
+ // Read records serially
|
|
|
+ callables.forEach(e -> {
|
|
|
+ try {
|
|
|
+ e.call();
|
|
|
+ } catch (Exception ex) {
|
|
|
+ LOG.error("Failed to retrieve record using file operations.", ex);
|
|
|
+ throw new RuntimeException(ex);
|
|
|
+ }
|
|
|
+ });
|
|
|
}
|
|
|
} catch (Exception e) {
|
|
|
if (metrics != null) {
|
|
@@ -227,7 +270,37 @@ public abstract class StateStoreFileBaseImpl
|
|
|
if (metrics != null) {
|
|
|
metrics.addRead(monotonicNow() - start);
|
|
|
}
|
|
|
- return new QueryResult<T>(ret, getTime());
|
|
|
+ return new QueryResult<>(result, getTime());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the state store record from the given path (path/child) and add the record to the
|
|
|
+ * result list.
|
|
|
+ *
|
|
|
+ * @param clazz Class of the record.
|
|
|
+ * @param result The list of results record. The records would be added to it unless the given
|
|
|
+ * path represents old temp file.
|
|
|
+ * @param path The parent path.
|
|
|
+ * @param child The child path under the parent path. Both path and child completes the file
|
|
|
+ * location for the given record.
|
|
|
+ * @param <T> Record class of the records.
|
|
|
+ * @return Void.
|
|
|
+ * @throws IOException If the file read operation fails.
|
|
|
+ */
|
|
|
+ private <T extends BaseRecord> Void getRecordsFromFileAndRemoveOldTmpRecords(Class<T> clazz,
|
|
|
+ List<T> result, String path, String child) throws IOException {
|
|
|
+ String pathRecord = path + "/" + child;
|
|
|
+ if (child.endsWith(TMP_MARK)) {
|
|
|
+ LOG.debug("There is a temporary file {} in {}", child, path);
|
|
|
+ if (isOldTempRecord(child)) {
|
|
|
+ LOG.warn("Removing {} as it's an old temporary record", child);
|
|
|
+ remove(pathRecord);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ T record = getRecord(pathRecord, clazz);
|
|
|
+ result.add(record);
|
|
|
+ }
|
|
|
+ return null;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -260,23 +333,17 @@ public abstract class StateStoreFileBaseImpl
|
|
|
*/
|
|
|
private <T extends BaseRecord> T getRecord(
|
|
|
final String path, final Class<T> clazz) throws IOException {
|
|
|
- BufferedReader reader = getReader(path);
|
|
|
- try {
|
|
|
+ try (BufferedReader reader = getReader(path)) {
|
|
|
String line;
|
|
|
while ((line = reader.readLine()) != null) {
|
|
|
if (!line.startsWith("#") && line.length() > 0) {
|
|
|
try {
|
|
|
- T record = newRecord(line, clazz, false);
|
|
|
- return record;
|
|
|
+ return newRecord(line, clazz, false);
|
|
|
} catch (Exception ex) {
|
|
|
LOG.error("Cannot parse line {} in file {}", line, path, ex);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- } finally {
|
|
|
- if (reader != null) {
|
|
|
- reader.close();
|
|
|
- }
|
|
|
}
|
|
|
throw new IOException("Cannot read " + path + " for record " +
|
|
|
clazz.getSimpleName());
|
|
@@ -330,13 +397,12 @@ public abstract class StateStoreFileBaseImpl
|
|
|
record.setDateModified(this.getTime());
|
|
|
toWrite.put(recordPath, record);
|
|
|
} else if (errorIfExists) {
|
|
|
- LOG.error("Attempt to insert record {} that already exists",
|
|
|
- recordPath);
|
|
|
+ LOG.error("Attempt to insert record {} that already exists", recordPath);
|
|
|
if (metrics != null) {
|
|
|
metrics.addFailure(monotonicNow() - start);
|
|
|
}
|
|
|
return false;
|
|
|
- } else {
|
|
|
+ } else {
|
|
|
LOG.debug("Not updating {}", record);
|
|
|
}
|
|
|
} else {
|
|
@@ -345,36 +411,81 @@ public abstract class StateStoreFileBaseImpl
|
|
|
}
|
|
|
|
|
|
// Write the records
|
|
|
- boolean success = true;
|
|
|
- for (Entry<String, T> entry : toWrite.entrySet()) {
|
|
|
- String recordPath = entry.getKey();
|
|
|
- String recordPathTemp = recordPath + "." + now() + TMP_MARK;
|
|
|
- boolean recordWrittenSuccessfully = true;
|
|
|
- try (BufferedWriter writer = getWriter(recordPathTemp)) {
|
|
|
- T record = entry.getValue();
|
|
|
- String line = serializeString(record);
|
|
|
- writer.write(line);
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.error("Cannot write {}", recordPathTemp, e);
|
|
|
- recordWrittenSuccessfully = false;
|
|
|
- success = false;
|
|
|
+ final AtomicBoolean success = new AtomicBoolean(true);
|
|
|
+ final List<Callable<Void>> callables = new ArrayList<>();
|
|
|
+ toWrite.entrySet().forEach(entry -> callables.add(() -> writeRecordToFile(success, entry)));
|
|
|
+ if (this.concurrentStoreAccessPool != null) {
|
|
|
+ // Write records concurrently
|
|
|
+ List<Future<Void>> futures = null;
|
|
|
+ try {
|
|
|
+ futures = this.concurrentStoreAccessPool.invokeAll(callables);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ success.set(false);
|
|
|
+ LOG.error("Failed to put record concurrently.", e);
|
|
|
}
|
|
|
- // Commit
|
|
|
- if (recordWrittenSuccessfully && !rename(recordPathTemp, recordPath)) {
|
|
|
- LOG.error("Failed committing record into {}", recordPath);
|
|
|
- success = false;
|
|
|
+ if (futures != null) {
|
|
|
+ for (Future<Void> future : futures) {
|
|
|
+ try {
|
|
|
+ future.get();
|
|
|
+ } catch (InterruptedException | ExecutionException e) {
|
|
|
+ success.set(false);
|
|
|
+ LOG.error("Failed to retrieve results from concurrent record put runs.", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
+ } else {
|
|
|
+ // Write records serially
|
|
|
+ callables.forEach(callable -> {
|
|
|
+ try {
|
|
|
+ callable.call();
|
|
|
+ } catch (Exception e) {
|
|
|
+ success.set(false);
|
|
|
+ LOG.error("Failed to put record.", e);
|
|
|
+ }
|
|
|
+ });
|
|
|
}
|
|
|
|
|
|
long end = monotonicNow();
|
|
|
if (metrics != null) {
|
|
|
- if (success) {
|
|
|
+ if (success.get()) {
|
|
|
metrics.addWrite(end - start);
|
|
|
} else {
|
|
|
metrics.addFailure(end - start);
|
|
|
}
|
|
|
}
|
|
|
- return success;
|
|
|
+ return success.get();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Writes the state store record to the file. At first, the record is written to a temp location
|
|
|
+ * and then later renamed to the final location that is passed with the entry key.
|
|
|
+ *
|
|
|
+ * @param success The atomic boolean that gets updated to false if the file write operation fails.
|
|
|
+ * @param entry The entry of the record path and the state store record to be written to the file
|
|
|
+ * by first writing to a temp location and then renaming it to the record path.
|
|
|
+ * @param <T> Record class of the records.
|
|
|
+ * @return Void.
|
|
|
+ */
|
|
|
+ private <T extends BaseRecord> Void writeRecordToFile(AtomicBoolean success,
|
|
|
+ Entry<String, T> entry) {
|
|
|
+ String recordPath = entry.getKey();
|
|
|
+ String recordPathTemp = recordPath + "." + now() + TMP_MARK;
|
|
|
+ boolean recordWrittenSuccessfully = true;
|
|
|
+ try (BufferedWriter writer = getWriter(recordPathTemp)) {
|
|
|
+ T record = entry.getValue();
|
|
|
+ String line = serializeString(record);
|
|
|
+ writer.write(line);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Cannot write {}", recordPathTemp, e);
|
|
|
+ recordWrittenSuccessfully = false;
|
|
|
+ success.set(false);
|
|
|
+ }
|
|
|
+ // Commit
|
|
|
+ if (recordWrittenSuccessfully && !rename(recordPathTemp, recordPath)) {
|
|
|
+ LOG.error("Failed committing record into {}", recordPath);
|
|
|
+ success.set(false);
|
|
|
+ }
|
|
|
+ return null;
|
|
|
}
|
|
|
|
|
|
@Override
|