Skip to content
Merged
Show file tree
Hide file tree
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 @@ -78,9 +78,10 @@ public long getEstimatedTotalKeys() throws RocksDBException {

try (ManagedOptions options = new ManagedOptions()) {
for (String sstFile : sstFiles) {
SstFileReader fileReader = new SstFileReader(options);
fileReader.open(sstFile);
estimatedSize += fileReader.getTableProperties().getNumEntries();
try (SstFileReader fileReader = new SstFileReader(options)) {
fileReader.open(sstFile);
estimatedSize += fileReader.getTableProperties().getNumEntries();
}
}
}
estimatedTotalKeys = estimatedSize;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -629,16 +629,18 @@ private long getSSTFileSummary(String filename)
filename += SST_FILE_EXTENSION;
}

Options option = new Options();
SstFileReader reader = new SstFileReader(option);
try (
ManagedOptions option = new ManagedOptions();
SstFileReader reader = new SstFileReader(option)) {

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

TableProperties properties = reader.getTableProperties();
if (LOG.isDebugEnabled()) {
LOG.debug("{} has {} keys", filename, properties.getNumEntries());
TableProperties properties = reader.getTableProperties();
if (LOG.isDebugEnabled()) {
LOG.debug("{} has {} keys", filename, properties.getNumEntries());
}
return properties.getNumEntries();
}
return properties.getNumEntries();
}

private String getAbsoluteSstFilePath(String filename)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,12 @@
package org.apache.ozone.rocksdiff;

import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hdds.utils.db.managed.ManagedOptions;
import org.apache.hadoop.hdds.utils.db.managed.ManagedReadOptions;
import org.apache.hadoop.hdds.utils.db.managed.ManagedSstFileReader;
import org.apache.hadoop.hdds.utils.db.managed.ManagedSstFileReaderIterator;
import org.rocksdb.SstFileReader;
import org.rocksdb.TableProperties;
import org.rocksdb.Options;
import org.rocksdb.ReadOptions;
import org.rocksdb.RocksDBException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -87,16 +87,20 @@ public static void filterRelevantSstFiles(Set<String> inputFiles,

public static boolean doesSstFileContainKeyRange(String filepath,
Map<String, String> tableToPrefixMap) throws IOException {
try (ManagedSstFileReader sstFileReader = ManagedSstFileReader.managed(
new SstFileReader(new Options()))) {

try (
ManagedOptions options = new ManagedOptions();
ManagedSstFileReader sstFileReader = ManagedSstFileReader.managed(new SstFileReader(options))) {
sstFileReader.get().open(filepath);
TableProperties properties = sstFileReader.get().getTableProperties();
String tableName = new String(properties.getColumnFamilyName(), UTF_8);
if (tableToPrefixMap.containsKey(tableName)) {
String prefix = tableToPrefixMap.get(tableName);
try (ManagedSstFileReaderIterator iterator =
ManagedSstFileReaderIterator.managed(sstFileReader.get()
.newIterator(new ReadOptions()))) {

try (
ManagedReadOptions readOptions = new ManagedReadOptions();
ManagedSstFileReaderIterator iterator = ManagedSstFileReaderIterator.managed(
sstFileReader.get().newIterator(readOptions))) {
iterator.get().seek(prefix.getBytes(UTF_8));
String seekResultKey = new String(iterator.get().key(), UTF_8);
return seekResultKey.startsWith(prefix);
Expand Down