Skip to content

Commit

Permalink
HDDS-10783. Close SstFileReaderIterator in RocksDBCheckpointDiffer (a…
Browse files Browse the repository at this point in the history
  • Loading branch information
hemantk-12 authored May 1, 2024
1 parent 78a7e7a commit a658802
Showing 1 changed file with 19 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,8 @@
import org.apache.hadoop.hdds.utils.db.managed.ManagedReadOptions;
import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
import org.apache.hadoop.hdds.utils.db.managed.ManagedSstFileReader;
import org.apache.hadoop.hdds.utils.db.managed.ManagedSstFileReaderIterator;
import org.apache.ozone.compaction.log.CompactionFileInfo;
import org.apache.ozone.compaction.log.CompactionLogEntry;
import org.apache.ozone.rocksdb.util.RdbUtil;
Expand All @@ -59,7 +61,6 @@
import org.rocksdb.RocksDB;
import org.rocksdb.RocksDBException;
import org.rocksdb.SstFileReader;
import org.rocksdb.SstFileReaderIterator;
import org.rocksdb.TableProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -629,13 +630,12 @@ private long getSSTFileSummary(String filename)
filename += SST_FILE_EXTENSION;
}

try (
ManagedOptions option = new ManagedOptions();
SstFileReader reader = new SstFileReader(option)) {
try (ManagedOptions option = new ManagedOptions();
ManagedSstFileReader reader = ManagedSstFileReader.managed(new SstFileReader(option))) {

reader.open(getAbsoluteSstFilePath(filename));
reader.get().open(getAbsoluteSstFilePath(filename));

TableProperties properties = reader.getTableProperties();
TableProperties properties = reader.get().getTableProperties();
if (LOG.isDebugEnabled()) {
LOG.debug("{} has {} keys", filename, properties.getNumEntries());
}
Expand Down Expand Up @@ -1575,18 +1575,19 @@ private CompactionFileInfo toFileInfo(String sstFile,
CompactionFileInfo.Builder fileInfoBuilder =
new CompactionFileInfo.Builder(fileName);

try (SstFileReader fileReader = new SstFileReader(options)) {
fileReader.open(sstFile);
String columnFamily = StringUtils.bytes2String(
fileReader.getTableProperties().getColumnFamilyName());
SstFileReaderIterator iterator = fileReader.newIterator(readOptions);
iterator.seekToFirst();
String startKey = StringUtils.bytes2String(iterator.key());
iterator.seekToLast();
String endKey = StringUtils.bytes2String(iterator.key());
fileInfoBuilder.setStartRange(startKey)
.setEndRange(endKey)
.setColumnFamily(columnFamily);
try (ManagedSstFileReader fileReader = ManagedSstFileReader.managed(new SstFileReader(options))) {
fileReader.get().open(sstFile);
String columnFamily = StringUtils.bytes2String(fileReader.get().getTableProperties().getColumnFamilyName());
try (ManagedSstFileReaderIterator iterator =
ManagedSstFileReaderIterator.managed(fileReader.get().newIterator(readOptions))) {
iterator.get().seekToFirst();
String startKey = StringUtils.bytes2String(iterator.get().key());
iterator.get().seekToLast();
String endKey = StringUtils.bytes2String(iterator.get().key());
fileInfoBuilder.setStartRange(startKey)
.setEndRange(endKey)
.setColumnFamily(columnFamily);
}
} catch (RocksDBException rocksDBException) {
// Ideally it should not happen. If it does just log the exception.
// And let the compaction complete without the exception.
Expand Down

0 comments on commit a658802

Please sign in to comment.