|
45 | 45 | import org.apache.hadoop.hdds.utils.db.managed.ManagedReadOptions; |
46 | 46 | import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; |
47 | 47 | import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB; |
| 48 | +import org.apache.hadoop.hdds.utils.db.managed.ManagedSstFileReader; |
| 49 | +import org.apache.hadoop.hdds.utils.db.managed.ManagedSstFileReaderIterator; |
48 | 50 | import org.apache.ozone.compaction.log.CompactionFileInfo; |
49 | 51 | import org.apache.ozone.compaction.log.CompactionLogEntry; |
50 | 52 | import org.apache.ozone.rocksdb.util.RdbUtil; |
|
59 | 61 | import org.rocksdb.RocksDB; |
60 | 62 | import org.rocksdb.RocksDBException; |
61 | 63 | import org.rocksdb.SstFileReader; |
62 | | -import org.rocksdb.SstFileReaderIterator; |
63 | 64 | import org.rocksdb.TableProperties; |
64 | 65 | import org.slf4j.Logger; |
65 | 66 | import org.slf4j.LoggerFactory; |
@@ -629,13 +630,12 @@ private long getSSTFileSummary(String filename) |
629 | 630 | filename += SST_FILE_EXTENSION; |
630 | 631 | } |
631 | 632 |
|
632 | | - try ( |
633 | | - ManagedOptions option = new ManagedOptions(); |
634 | | - SstFileReader reader = new SstFileReader(option)) { |
| 633 | + try (ManagedOptions option = new ManagedOptions(); |
| 634 | + ManagedSstFileReader reader = ManagedSstFileReader.managed(new SstFileReader(option))) { |
635 | 635 |
|
636 | | - reader.open(getAbsoluteSstFilePath(filename)); |
| 636 | + reader.get().open(getAbsoluteSstFilePath(filename)); |
637 | 637 |
|
638 | | - TableProperties properties = reader.getTableProperties(); |
| 638 | + TableProperties properties = reader.get().getTableProperties(); |
639 | 639 | if (LOG.isDebugEnabled()) { |
640 | 640 | LOG.debug("{} has {} keys", filename, properties.getNumEntries()); |
641 | 641 | } |
@@ -1575,18 +1575,19 @@ private CompactionFileInfo toFileInfo(String sstFile, |
1575 | 1575 | CompactionFileInfo.Builder fileInfoBuilder = |
1576 | 1576 | new CompactionFileInfo.Builder(fileName); |
1577 | 1577 |
|
1578 | | - try (SstFileReader fileReader = new SstFileReader(options)) { |
1579 | | - fileReader.open(sstFile); |
1580 | | - String columnFamily = StringUtils.bytes2String( |
1581 | | - fileReader.getTableProperties().getColumnFamilyName()); |
1582 | | - SstFileReaderIterator iterator = fileReader.newIterator(readOptions); |
1583 | | - iterator.seekToFirst(); |
1584 | | - String startKey = StringUtils.bytes2String(iterator.key()); |
1585 | | - iterator.seekToLast(); |
1586 | | - String endKey = StringUtils.bytes2String(iterator.key()); |
1587 | | - fileInfoBuilder.setStartRange(startKey) |
1588 | | - .setEndRange(endKey) |
1589 | | - .setColumnFamily(columnFamily); |
| 1578 | + try (ManagedSstFileReader fileReader = ManagedSstFileReader.managed(new SstFileReader(options))) { |
| 1579 | + fileReader.get().open(sstFile); |
| 1580 | + String columnFamily = StringUtils.bytes2String(fileReader.get().getTableProperties().getColumnFamilyName()); |
| 1581 | + try (ManagedSstFileReaderIterator iterator = |
| 1582 | + ManagedSstFileReaderIterator.managed(fileReader.get().newIterator(readOptions))) { |
| 1583 | + iterator.get().seekToFirst(); |
| 1584 | + String startKey = StringUtils.bytes2String(iterator.get().key()); |
| 1585 | + iterator.get().seekToLast(); |
| 1586 | + String endKey = StringUtils.bytes2String(iterator.get().key()); |
| 1587 | + fileInfoBuilder.setStartRange(startKey) |
| 1588 | + .setEndRange(endKey) |
| 1589 | + .setColumnFamily(columnFamily); |
| 1590 | + } |
1590 | 1591 | } catch (RocksDBException rocksDBException) { |
1591 | 1592 | // Ideally it should not happen. If it does just log the exception. |
1592 | 1593 | // And let the compaction complete without the exception. |
|
0 commit comments