Skip to content
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-10783. Close SstFileReaderIterator in RocksDBCheckpointDiffer #6616

Merged
merged 2 commits into from
May 1, 2024
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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