LuciferYang commented on code in PR #43502:
URL: https://github.com/apache/spark/pull/43502#discussion_r1373038414
##########
common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java:
##########
@@ -182,23 +189,21 @@ public boolean skip(long n) {
@Override
public synchronized void close() throws IOException {
- db.notifyIteratorClosed(this);
if (!closed) {
- it.close();
- closed = true;
- next = null;
+ try {
+ cleanable.clean();
Review Comment:
@mridulm Are you referring to the addition of `synchronized
(rocksDB.getRocksDB())` in `cleanable.clean()` compared to before? Perhaps we
can let the `close()` method still use the original logic as follows:
1. Add a state variable for ResourceCleaner and let RocksDB/LevelDBIterator
hold an instance of ResourceCleaner, like
```java
private final AtomicBoolean status = new AtomicBoolean(true);
void statusToFalse() {
status.set(false);
}
@Override
public void run() {
if (status.compareAndSet(true, false)) {
....
}
}
```
2. Add a method to RocksDB/LevelDBIterator to prevent ResourceCleaner from
actually releasing resources, like
```java
private void cancelXXX() {
this.resourceCleaner.statusToFalse();
this.cleanable.clean();
}
```
3. change the `close()` to
```java
public synchronized void close() throws IOException {
db.notifyIteratorClosed(this);
if (!closed) {
it.close();
closed = true;
next = null;
this.cancelXXX();
}
}
```
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]