-
Notifications
You must be signed in to change notification settings - Fork 594
HDDS-12607. Parallelize recon tasks to speed up OM rocksdb reading tasks #9243
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HDDS-12607. Parallelize recon tasks to speed up OM rocksdb reading tasks #9243
Conversation
|
share some performance data as seen improved due to parallel reprocess |
sumitagrawl
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@ArafatKhan2198 Due to recent features and old issue, data is not updated with synchronized way. Might be we can change model,
Producer-- (table iterator with decode) ==> consumer (single thread that will update NSSummary) instead of having multiple times lock take / release and avoid starvation.
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java
Outdated
Show resolved
Hide resolved
...one/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java
Outdated
Show resolved
Hide resolved
...one/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java
Outdated
Show resolved
Hide resolved
...-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java
Outdated
Show resolved
Hide resolved
devmadhuu
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @ArafatKhan2198 for the patch. Given some comments, Also the PR description and objective mentioned is confusing.
We are not parallelizing the recon tasks, rather we are improving all recon tasks's full snapshot reprocessing where we are processing all sst files in the respective rocksDB table needed for that task in concurrent fashion rather iterating the respective table's DB keys in sequential fashion. Pls correct the PR description to avoid any confusion. Recon tasks as such in itself are already being running in parallel before this PR change.
...n/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java
Show resolved
Hide resolved
...on/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java
Outdated
Show resolved
Hide resolved
devmadhuu
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@ArafatKhan2198 , Pls check , few more comments.
...n/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java
Show resolved
Hide resolved
| this.maxIteratorTasks = 2 * iteratorCount; | ||
| this.maxWorkerTasks = workerCount * 2; | ||
| this.iteratorExecutor = new ThreadPoolExecutor(iteratorCount, iteratorCount, 1, TimeUnit.MINUTES, | ||
| new ArrayBlockingQueue<>(iteratorCount * 2)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This has a potential issue where if queue fills up, there is no RejectionPolicy defined and RejectedExecutionException lead Recon to crash
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Implemented with CallerRunsPolicy. Added rejection policy to both thread pools:
|
|
||
| try (ParallelTableIteratorOperation<String, OmKeyInfo> keyIter = | ||
| new ParallelTableIteratorOperation<>(omMetadataManager, omKeyInfoTable, | ||
| StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, 100000)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Where the flush logic gone ? Why removed ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Had removed it by mistake, will add it back
| reconContainerMetadataManager)) { | ||
| LOG.error("Failed to flush container key data for {}", taskName); | ||
| return false; | ||
| ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Use fair locking, else thread starvation issue can happen.
- ReentrantReadWriteLock is non-fair by default
- With 20 worker threads continuously acquiring read locks, write lock requests can be starved indefinitely
- When containerKeyMap reaches threshold, threads need write lock to flush, but may wait forever
Scenario:
Thread 1: [Read Lock] -> processing key 1
Thread 2: [Read Lock] -> processing key 2
Thread 3: [Read Lock] -> processing key 3
...
Thread 20: [Read Lock] -> processing key 20
Thread 21: [Waiting Write Lock] <- BLOCKED! (containerKeyMap full, needs to flush)
Thread 1: [Read Lock] -> processing key 21 (lock released and reacquired)
Thread 2: [Read Lock] -> processing key 22
... Thread 21 STILL WAITING
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Evaluated but decided against due to performance impact. I initially implemented fair locking (new ReentrantReadWriteLock(true)), but observed 60%+ performance degradation in testing (throughput dropped from ~60K keys/sec to ~7K keys/sec).
Alternative mitigation implemented:
AtomicBoolean isFlushingInProgress = new AtomicBoolean(false);
if (map.size() >= threshold && isFlushingInProgress.compareAndSet(false, true)) {
// Only ONE thread attempts flush
}
This flag-based coordination prevents the mass queueing scenario while maintaining performance. In our 8.7M key workload, flushing happens ~58 times total (every 150K keys), making starvation statistically unlikely. Open to revisiting if starvation is observed in production.
...-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java
Outdated
Show resolved
Hide resolved
...-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java
Outdated
Show resolved
Hide resolved
… for fileSizeCountTask, and made ReentrantReadWriteLock fair
|
Thank you for the thorough review! @devmadhuu |
|
I tested on an OM DB with 8.7 million keys and 5.6 million directories, totaling about 1.2 TB of data.
I have also attached the recon logs for verification - recon.log |
devmadhuu
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@ArafatKhan2198 thanks for improving the patch. Pls see some more comments.
...-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java
Outdated
Show resolved
Hide resolved
| try { | ||
| lock.writeLock().lock(); | ||
| try { | ||
| if (!checkAndCallFlushToDB(containerKeyMap, containerKeyFlushToDBMaxThreshold, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is triple locking here which increases contention and not needed. checkAndCallFlushToDB method also using static synchronization.
- write lock → static monitor → RocksDB locks
- Increases contention
- Increases latency
- Increases likelihood of deadlock
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks, I have made the change.
...n/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java
Outdated
Show resolved
Hide resolved
| // Double-check after acquiring write lock | ||
| if (fileSizeCountMap.size() >= FLUSH_THRESHOLD) { | ||
| LOG.debug("Flushing {} accumulated counts to RocksDB for {}", fileSizeCountMap.size(), taskName); | ||
| writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Now this is being called by multiple threads in hot path and in this method, for every key, we are doing read DB operation, this could be performance issue,
Long existingCount = reconFileMetadataManager.getFileSizeCount(key);
It is O(N) DB reads instead of O(1) batch operations
...p-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java
Outdated
Show resolved
Hide resolved
...-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java
Outdated
Show resolved
Hide resolved
…tHashMap<Long,AtomicLong> for container counts
devmadhuu
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @ArafatKhan2198 for improving the patch, Need to properly check the error handling behavior and few more comments. Please check carefully.
Please add some test coverage:
- Concurrent FSO + OBS task execution
- Thread pool exhaustion
Below can be done in separate PR, you can mention the JIRA for this comment here.
Add metrics for parallel processing
- Worker thread utilization
- Lock wait times
- Queue depths
...on/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java
Outdated
Show resolved
Hide resolved
...on/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java
Outdated
Show resolved
Hide resolved
...p-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java
Outdated
Show resolved
Hide resolved
...p-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java
Outdated
Show resolved
Hide resolved
...p-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java
Outdated
Show resolved
Hide resolved
...n/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java
Outdated
Show resolved
Hide resolved
...n/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java
Show resolved
Hide resolved
...n/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java
Show resolved
Hide resolved
...n/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java
Show resolved
Hide resolved
...n/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java
Show resolved
Hide resolved
sumitagrawl
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@swamirishi there are few comments over parallel iterator correctness when filter is present, please check
cc: @ArafatKhan2198
...n/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java
Show resolved
Hide resolved
...n/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java
Show resolved
Hide resolved
...n/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java
Show resolved
Hide resolved
| } | ||
| if (!keyValues.isEmpty()) { | ||
| waitForQueueSize(workerFutures, maxWorkerTasks - 10); | ||
| workerFutures.add(valueExecutors.submit(() -> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
workerFutures will wait for the batch being queued. May have some delay in lesser number of splits. Otherway is,
Blocking-workerQueue --
Producer (multi-threaded, ie iterator-executor): kv is added as soon identified
consumer: Set of thread waiting on queue to retrive and process
End point: set flag when iterator is completed
This might be simpler implementation to avoid intermediate wait.
|
|
||
| public static final AtomicBoolean CONTAINER_KEY_TABLES_TRUNCATED = new AtomicBoolean(false); | ||
|
|
||
| public static final AtomicBoolean CONTAINER_KEY_COUNT_MAP_INITIALIZED = new AtomicBoolean(false); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
no need this Atomic boolean.
| * cross-task data corruption where FSO and OBS tasks overwrite each other's counts. | ||
| */ | ||
| private static void initializeSharedContainerCountMapIfNeeded(String taskName) { | ||
| synchronized (SHARED_MAP_LOCK) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
init of map and clear can be done using truncate_lock, rename this lock.
| try { | ||
| // No deleted container list needed since "reprocess" only has put operations | ||
| writeToTheDB(containerKeyMap, containerKeyCountMap, Collections.emptyList(), reconContainerMetadataManager); | ||
| writeToTheDB(localContainerKeyMap, sharedContainerKeyCountMap, Collections.emptyList(), reconContainerMetadataManager); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
only last task getting executed need flush and cleanup map, else data can got wrong.
| writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); | ||
| long writeStartTime = Time.monotonicNow(); | ||
| // Acquire GLOBAL lock (cross-task) before writing to DB | ||
| FILE_COUNT_WRITE_LOCK.writeLock().lock(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
FILE_COUNT_LOCK needs to be local lock
| * Process table in parallel using multiple iterators and workers. | ||
| * Only for tables with String keys. | ||
| */ | ||
| private void processTableInParallel(String tableName, Table<?, ?> table, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
no need pass table here, get table with StringCodec as done in try-catch and use same if required other place
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Implemented. Removed table parameter and get table inside method with StringCodec as suggested. Method now takes only tableName and omMetadataManager.
| * Calculate logging threshold based on table size. | ||
| * Logs progress every 1% of total keys, minimum 1. | ||
| */ | ||
| private long calculateLoggingThreshold(Table<?, ?> table) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can pass estimatedCount directly.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Implemented. Changed method to take long estimatedCount instead of Table.
| * Process table sequentially using raw iterator (no type assumptions). | ||
| * Used for tables with non-String keys or as fallback. | ||
| */ | ||
| private void processTableSequentially(String tableName, Table<?, ?> table) throws IOException { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we can get table inside only with keyIterator(), no need value iterator.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Implemented. Changed to use table.keyIterator() instead of full iterator.
| replicatedSizeMap.put(getReplicatedSizeKeyFromTable(tableName), | ||
| details.getRight()); | ||
| Table<String, ?> stringTable = (Table<String, ?>) table; | ||
| try (TableIterator<String, ? extends Table.KeyValue<String, ?>> iterator = stringTable.iterator()) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we can remove table object and getTableSizeAndCount can get table object and return count
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Implemented. Modified OmTableHandler interface and both implementations (OpenKeysInsightHandler, DeletedKeysInsightHandler) to take tableName and omMetadataManager instead of iterator. Handlers now get table and create iterators themselves.
| if (myMap.size() >= PER_WORKER_THRESHOLD) { | ||
| synchronized (flushLock) { | ||
| writeCountsToDB(myMap, reconFileMetadataManager); | ||
| myMap.clear(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
rename myMap variable as fileSizeCountMap
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Implemented. Renamed myMap to workerFileSizeCountMap for clarity
| SHARED_CONTAINER_KEY_COUNT_MAP.clear(); | ||
|
|
||
| // Step 3: Initialize reference counter (2 tasks: FSO + OBS) | ||
| ACTIVE_TASK_COUNT.set(2); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
need increment in synchronized_lock by one every task, and decrement in same lock when finish
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Implemented
| Function<Table.KeyValue<String, OmKeyInfo>, Void> kvOperation = kv -> { | ||
| try { | ||
| // Get or create this worker's private local map using thread ID | ||
| Map<ContainerKeyPrefix, Integer> myLocalMap = allLocalMaps.computeIfAbsent( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
rename myLocalMap to containerKeyPrefixMap
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Renamed myLocalMap to containerKeyPrefixMap for better clarity.
| if (!checkAndCallFlushToDB(containerKeyMap, containerKeyFlushToDBMaxThreshold, | ||
| // Flush this worker's map when it reaches threshold | ||
| if (myLocalMap.size() >= PER_WORKER_THRESHOLD) { | ||
| synchronized (flushLock) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we do not need lock as containerKeyPrefix is unique, added to table.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks I have now Removed flushLock from periodic flush as each worker flushes unique ContainerKeyPrefix values (containerId + keyPrefix + keyVersion combinations).
...-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java
Show resolved
Hide resolved
| long totalContainers = SHARED_CONTAINER_KEY_COUNT_MAP.size(); | ||
|
|
||
| // Final flush: Shared container count map | ||
| if (!flushAndCommitContainerKeyInfoToDB(Collections.emptyMap(), SHARED_CONTAINER_KEY_COUNT_MAP, reconContainerMetadataManager)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
flush to be done at last, else can have twice flush.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Implemented. Moved shared container count map flush inside the if (remainingTasks == 0) block. Now only the last completing task flushes the shared map once, preventing duplicate flushes.
sumitagrawl
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Given few comment
...ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java
Outdated
Show resolved
Hide resolved
...op-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java
Outdated
Show resolved
Hide resolved
...-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java
Outdated
Show resolved
Hide resolved
...-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java
Show resolved
Hide resolved
…NT leak on initialization exception.
...-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java
Show resolved
Hide resolved
...ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java
Outdated
Show resolved
Hide resolved
sumitagrawl
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
|
Got a green run for CI - |
Summary
Implemented parallel table iteration in Recon snapshot reprocessing to significantly improve performance. Instead of reading tables sequentially with a single iterator, this change uses multiple iterator threads to read different RocksDB SST file segments concurrently, with worker threads processing batches in parallel.
What Changed
ContainerKeyMapperHelper- Parallel iteration for FSO and OBS tablesFileSizeCountTaskHelper- Parallel iteration for FSO and OBS tablesOmTableInsightTask- Parallel counting for simple tables, sequential withkeyIterator()for tables with non-String keys (dTokenTable, s3SecretTable)Key Implementation Details
LiveFileMetaDataboundaries, creating batches. 20 worker threads (or 2 for simple counting) process these batches.ConcurrentHashMap.computeIfAbsent(Thread.currentThread().getId())- no read/write locks needed for map updatesContainerKeyMapperTaskuses sharedConcurrentHashMap<Long, AtomicLong>because FSO and OBS tasks write to the same container IDs.FileSizeCountTaskdoesn't need shared state (FSO and OBS write different bucket keys).ozone.recon.task.reprocess.max.iterators= 5ozone.recon.task.reprocess.max.workers= 20ozone.recon.task.reprocess.max.keys.in.memory= 2000ozone.recon.filesizecount.flush.db.max.threshold= 200000What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/HDDS-12607
How was this patch tested?
Performance Results
Tested on OM RocksDB with 104M keys and 5.4M directories: