From 42bb05d2558052c436a6d1dd309104103bd3e5a9 Mon Sep 17 00:00:00 2001 From: Hemant Kumar Date: Mon, 28 Aug 2023 22:39:10 -0700 Subject: [PATCH 1/6] Missing Sst file fix --- .../hadoop/hdds/utils/db/RocksDatabase.java | 69 +++--- .../src/main/proto/hdds.proto | 15 +- .../ozone/rocksdiff/CompactionLogEntry.java | 179 ++++++++++++++ .../ozone/rocksdiff/CompactionNode.java | 32 ++- .../org/apache/ozone/rocksdiff/FileInfo.java | 77 ++++++ .../rocksdiff/RocksDBCheckpointDiffer.java | 229 ++++++++++++------ .../ozone/rocksdiff/RocksDiffUtils.java | 6 +- .../TestRocksDBCheckpointDiffer.java | 23 +- .../ozone/rocksdiff/TestRocksDiffUtils.java | 57 +++++ .../hadoop/ozone/freon/TestOMSnapshotDAG.java | 22 +- .../hadoop/ozone/om/SstFilteringService.java | 63 +---- .../om/snapshot/SnapshotDiffManager.java | 8 +- .../ozone/om/snapshot/SnapshotUtils.java | 33 +++ .../ozone/om/TestSstFilteringService.java | 24 -- .../om/snapshot/TestSnapshotDiffManager.java | 9 +- 15 files changed, 621 insertions(+), 225 deletions(-) create mode 100644 hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java create mode 100644 hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/FileInfo.java create mode 100644 hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDiffUtils.java diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java index 8ff4317b05a9..f49bd0768887 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java @@ -18,9 +18,7 @@ package org.apache.hadoop.hdds.utils.db; import com.google.common.annotations.VisibleForTesting; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.StringUtils; -import org.apache.hadoop.hdds.utils.BooleanTriFunction; import org.apache.hadoop.hdds.utils.HddsServerUtil; import org.apache.hadoop.hdds.utils.db.managed.ManagedCheckpoint; import org.apache.hadoop.hdds.utils.db.managed.ManagedColumnFamilyOptions; @@ -35,6 +33,7 @@ import org.apache.hadoop.hdds.utils.db.managed.ManagedTransactionLogIterator; import org.apache.hadoop.hdds.utils.db.managed.ManagedWriteBatch; import org.apache.hadoop.hdds.utils.db.managed.ManagedWriteOptions; +import org.apache.ozone.rocksdiff.RocksDiffUtils; import org.rocksdb.ColumnFamilyDescriptor; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.Holder; @@ -48,7 +47,6 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; @@ -65,6 +63,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.hadoop.hdds.StringUtils.bytes2String; import static org.apache.hadoop.hdds.utils.db.managed.ManagedColumnFamilyOptions.closeDeeply; import static org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator.managed; @@ -611,7 +610,7 @@ private ColumnFamilyHandle getColumnFamilyHandle(String cfName) assertClose(); for (ColumnFamilyHandle cf : getCfHandleMap().get(db.get().getName())) { try { - String table = new String(cf.getName(), StandardCharsets.UTF_8); + String table = new String(cf.getName(), UTF_8); if (cfName.equals(table)) { return cf; } @@ -949,46 +948,42 @@ private int getLastLevel() throws IOException { /** * Deletes sst files which do not correspond to prefix * for given table. - * @param prefixPairs, a list of pair (TableName,prefixUsed). + * @param prefixPairs, a map of TableName to prefixUsed. */ - public void deleteFilesNotMatchingPrefix( - List> prefixPairs, - BooleanTriFunction filterFunction) + public void deleteFilesNotMatchingPrefix(Map prefixPairs) throws IOException, RocksDBException { assertClose(); for (LiveFileMetaData liveFileMetaData : getSstFileList()) { String sstFileColumnFamily = new String(liveFileMetaData.columnFamilyName(), - StandardCharsets.UTF_8); + UTF_8); int lastLevel = getLastLevel(); - for (Pair prefixPair : prefixPairs) { - String columnFamily = prefixPair.getKey(); - String prefixForColumnFamily = prefixPair.getValue(); - if (!sstFileColumnFamily.equals(columnFamily)) { - continue; - } - // RocksDB #deleteFile API allows only to delete the last level of - // SST Files. Any level < last level won't get deleted and - // only last file of level 0 can be deleted - // and will throw warning in the rocksdb manifest. - // Instead, perform the level check here - // itself to avoid failed delete attempts for lower level files. - if (liveFileMetaData.level() != lastLevel || lastLevel == 0) { - continue; - } - String firstDbKey = - new String(liveFileMetaData.smallestKey(), StandardCharsets.UTF_8); - String lastDbKey = - new String(liveFileMetaData.largestKey(), StandardCharsets.UTF_8); - boolean isKeyWithPrefixPresent = - filterFunction.apply(firstDbKey, lastDbKey, prefixForColumnFamily); - if (!isKeyWithPrefixPresent) { - LOG.info("Deleting sst file {} corresponding to column family" - + " {} from db: {}", liveFileMetaData.fileName(), - StringUtils.bytes2String(liveFileMetaData.columnFamilyName()), - db.get().getName()); - db.deleteFile(liveFileMetaData); - } + + if (!prefixPairs.containsKey(sstFileColumnFamily)) { + continue; + } + + // RocksDB #deleteFile API allows only to delete the last level of + // SST Files. Any level < last level won't get deleted and + // only last file of level 0 can be deleted + // and will throw warning in the rocksdb manifest. + // Instead, perform the level check here + // itself to avoid failed delete attempts for lower level files. + if (liveFileMetaData.level() != lastLevel || lastLevel == 0) { + continue; + } + + String prefixForColumnFamily = prefixPairs.get(sstFileColumnFamily); + String firstDbKey = new String(liveFileMetaData.smallestKey(), UTF_8); + String lastDbKey = new String(liveFileMetaData.largestKey(), UTF_8); + boolean isKeyWithPrefixPresent = RocksDiffUtils.isKeyWithPrefixPresent( + prefixForColumnFamily, firstDbKey, lastDbKey); + if (!isKeyWithPrefixPresent) { + LOG.info("Deleting sst file {} corresponding to column family" + + " {} from db: {}", liveFileMetaData.fileName(), + StringUtils.bytes2String(liveFileMetaData.columnFamilyName()), + db.get().getName()); + db.deleteFile(liveFileMetaData); } } } diff --git a/hadoop-hdds/interface-client/src/main/proto/hdds.proto b/hadoop-hdds/interface-client/src/main/proto/hdds.proto index 1479daa1c667..8090c62d8248 100644 --- a/hadoop-hdds/interface-client/src/main/proto/hdds.proto +++ b/hadoop-hdds/interface-client/src/main/proto/hdds.proto @@ -479,4 +479,17 @@ message DeletedBlocksTransactionInfo { optional int64 containerID = 2; repeated int64 localID = 3; optional int32 count = 4; -} \ No newline at end of file +} + +message FileInfoProto { + optional string fileName = 1; + optional string startKey = 2; + optional string endKey = 3; + optional string columnFamily = 4; +} + +message CompactionLogEntryProto { + optional uint64 dbSequenceNumber = 1; + repeated FileInfoProto inputFileIntoList = 2; + repeated FileInfoProto outputFileIntoList = 3; +} diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java new file mode 100644 index 000000000000..85f61bd4df7c --- /dev/null +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ozone.rocksdiff; + +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.commons.collections.CollectionUtils; +import org.apache.hadoop.hdds.StringUtils; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.CompactionLogEntryProto; +import org.apache.hadoop.hdds.utils.db.managed.ManagedOptions; +import org.apache.hadoop.hdds.utils.db.managed.ManagedReadOptions; +import org.rocksdb.RocksDBException; +import org.rocksdb.SstFileReader; +import org.rocksdb.SstFileReaderIterator; + +import java.util.ArrayList; +import java.util.Base64; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.SST_FILE_EXTENSION_LENGTH; + +/** + * Compaction log entry Dao to write to the compaction log file. + */ +public class CompactionLogEntry { + private final long dbSequenceNumber; + private final List inputFileInfoList; + private final List outputFileInfoList; + + public CompactionLogEntry(long dbSequenceNumber, + List inputFileInfoList, + List outputFileInfoList) { + this.dbSequenceNumber = dbSequenceNumber; + this.inputFileInfoList = inputFileInfoList; + this.outputFileInfoList = outputFileInfoList; + } + + public List getInputFileInfoList() { + return inputFileInfoList; + } + + public List getOutputFileInfoList() { + return outputFileInfoList; + } + + public long getDbSequenceNumber() { + return dbSequenceNumber; + } + + public CompactionLogEntryProto getProtobuf() { + CompactionLogEntryProto.Builder builder = CompactionLogEntryProto + .newBuilder() + .setDbSequenceNumber(dbSequenceNumber); + + if (inputFileInfoList != null) { + inputFileInfoList.forEach(fileInfo -> + builder.addInputFileIntoList(fileInfo.getProtobuf())); + } + + if (outputFileInfoList != null) { + outputFileInfoList.forEach(fileInfo -> + builder.addOutputFileIntoList(fileInfo.getProtobuf())); + } + + return builder.build(); + } + + public static CompactionLogEntry getFromProtobuf( + CompactionLogEntryProto proto) { + List inputFileInfo = proto.getInputFileIntoListList().stream() + .map(FileInfo::getFromProtobuf) + .collect(Collectors.toList()); + + List outputFileInfo = proto.getOutputFileIntoListList().stream() + .map(FileInfo::getFromProtobuf) + .collect(Collectors.toList()); + + + return new CompactionLogEntry(proto.getDbSequenceNumber(), + inputFileInfo, outputFileInfo); + } + + public String toEncodedString() { + // Encoding is used to deal with \n. Protobuf appends \n after each + // parameter. If ByteArray is simply converted to a string or + // protobuf.toString(), it will contain \n and will be added to the log. + // Which creates a problem while reading compaction logs. + // Compaction log lines are read sequentially assuming each line is one + // compaction log entry. + return Base64.getEncoder().encodeToString(getProtobuf().toByteArray()); + } + + public static CompactionLogEntry fromEncodedString(String string) { + try { + byte[] decode = Base64.getDecoder().decode(string); + return getFromProtobuf(CompactionLogEntryProto.parseFrom(decode)); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + } + + @Override + public String toString() { + return String.format("dbSequenceNumber: '%s', inputFileInfoList: '%s'," + + " outputFileInfoList: '%s',", dbSequenceNumber, inputFileInfoList, + outputFileInfoList); + } + + public static CompactionLogEntry fromCompactionFiles( + long dbSequenceNumber, + List inputFiles, + List outputFiles + ) { + + try (ManagedOptions options = new ManagedOptions(); + ManagedReadOptions readOptions = new ManagedReadOptions()) { + List inputFileInfos = convertFileInfo(inputFiles, options, + readOptions); + List outputFileInfos = convertFileInfo(outputFiles, options, + readOptions); + return new CompactionLogEntry(dbSequenceNumber, inputFileInfos, + outputFileInfos); + } + } + + private static List convertFileInfo( + List sstFiles, + ManagedOptions options, + ManagedReadOptions readOptions + ) { + if (CollectionUtils.isEmpty(sstFiles)) { + return Collections.emptyList(); + } + + final int fileNameOffset = sstFiles.get(0).lastIndexOf("/") + 1; + List response = new ArrayList<>(); + + for (String sstFile : sstFiles) { + String fileName = sstFile.substring(fileNameOffset, + sstFile.length() - SST_FILE_EXTENSION_LENGTH); + SstFileReader fileReader = new SstFileReader(options); + try { + 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()); + + FileInfo fileInfo = new FileInfo(fileName, startKey, endKey, + columnFamily); + response.add(fileInfo); + } catch (RocksDBException rocksDBException) { + throw new RuntimeException("Failed to read SST file: " + sstFile, + rocksDBException); + } + } + return response; + } +} diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionNode.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionNode.java index a7cfa27aaf28..a1faf2158173 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionNode.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionNode.java @@ -28,6 +28,9 @@ public class CompactionNode { private final long snapshotGeneration; private final long totalNumberOfKeys; private long cumulativeKeysReverseTraversal; + private final String startKey; + private final String endKey; + private final String columnFamily; /** * CompactionNode constructor. @@ -36,12 +39,17 @@ public class CompactionNode { * @param numKeys Number of keys in the SST * @param seqNum Snapshot generation (sequence number) */ - public CompactionNode(String file, String ssId, long numKeys, long seqNum) { - fileName = file; - snapshotId = ssId; - totalNumberOfKeys = numKeys; - snapshotGeneration = seqNum; - cumulativeKeysReverseTraversal = 0L; + public CompactionNode(String file, String ssId, long numKeys, long seqNum, + String keyStartRange, String keyEndRange, + String columnFamily) { + this.fileName = file; + this.snapshotId = ssId; + this.totalNumberOfKeys = numKeys; + this.snapshotGeneration = seqNum; + this.cumulativeKeysReverseTraversal = 0L; + this.startKey = keyStartRange; + this.endKey = keyEndRange; + this.columnFamily = columnFamily; } @Override @@ -77,4 +85,16 @@ public void setCumulativeKeysReverseTraversal( public void addCumulativeKeysReverseTraversal(long diff) { this.cumulativeKeysReverseTraversal += diff; } + + public String getStartKey() { + return startKey; + } + + public String getEndKey() { + return endKey; + } + + public String getColumnFamily() { + return columnFamily; + } } diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/FileInfo.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/FileInfo.java new file mode 100644 index 000000000000..bfa65534955c --- /dev/null +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/FileInfo.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ozone.rocksdiff; + +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; + +/** + * Dao to keep SST file information in the compaction log. + */ +public class FileInfo { + private final String fileName; + private final String startKey; + private final String endKey; + private final String columnFamily; + + public FileInfo(String fileName, + String startRange, + String endRange, + String columnFamily) { + this.fileName = fileName; + this.startKey = startRange; + this.endKey = endRange; + this.columnFamily = columnFamily; + } + + public String getFileName() { + return fileName; + } + + public String getStartKey() { + return startKey; + } + + public String getEndKey() { + return endKey; + } + + public String getColumnFamily() { + return columnFamily; + } + + public HddsProtos.FileInfoProto getProtobuf() { + return HddsProtos.FileInfoProto.newBuilder() + .setFileName(fileName) + .setStartKey(startKey) + .setEndKey(endKey) + .setColumnFamily(columnFamily) + .build(); + } + + public static FileInfo getFromProtobuf(HddsProtos.FileInfoProto proto) { + return new FileInfo(proto.getFileName(), proto.getStartKey(), + proto.getEndKey(), proto.getColumnFamily()); + } + + @Override + public String toString() { + return String.format("fileName: '%s', startKey: '%s', endKey: '%s'," + + " columnFamily: '%s'", fileName, startKey, endKey, columnFamily); + } +} diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java index fcc09eb28a10..b1a322597e8c 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java @@ -33,6 +33,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.utils.IOUtils; @@ -74,6 +75,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; import java.util.stream.Stream; + import org.apache.hadoop.ozone.lock.BootstrapStateHandler; import static java.nio.charset.StandardCharsets.UTF_8; @@ -138,6 +140,11 @@ public class RocksDBCheckpointDiffer implements AutoCloseable, */ private static final String COMPACTION_LOG_ENTRY_LINE_PREFIX = "C "; + /** + * Marks the beginning of a compaction log entry. + */ + private static final String COMPACTION_LOG_ENTRY_LINE_PREFIX_V1 = "C1 "; + /** * Prefix for the sequence number line when writing to compaction log * right after taking an Ozone snapshot. @@ -165,8 +172,7 @@ public class RocksDBCheckpointDiffer implements AutoCloseable, * to save space. */ static final String SST_FILE_EXTENSION = ".sst"; - private static final int SST_FILE_EXTENSION_LENGTH = - SST_FILE_EXTENSION.length(); + static final int SST_FILE_EXTENSION_LENGTH = SST_FILE_EXTENSION.length(); private static final int LONG_MAX_STR_LEN = String.valueOf(Long.MAX_VALUE).length(); @@ -305,6 +311,7 @@ private void createSstBackUpDir() { /** * Set the current compaction log filename with a given RDB sequence number. + * * @param latestSequenceNum latest sequence number of RDB. */ public void setCurrentCompactionLog(long latestSequenceNum) { @@ -548,6 +555,11 @@ public void onCompactionCompleted(RocksDB db, return; } + CompactionLogEntry compactionLogEntry = CompactionLogEntry + .fromCompactionFiles(db.getLatestSequenceNumber(), + compactionJobInfo.inputFiles(), + compactionJobInfo.outputFiles()); + final StringBuilder sb = new StringBuilder(); if (LOG.isDebugEnabled()) { @@ -558,38 +570,9 @@ public void onCompactionCompleted(RocksDB db, .append('\n'); } - // Mark the beginning of a compaction log - sb.append(COMPACTION_LOG_ENTRY_LINE_PREFIX); - sb.append(db.getLatestSequenceNumber()); - sb.append(SPACE_DELIMITER); - - // Trim DB path, only keep the SST file name - final int filenameOffset = - compactionJobInfo.inputFiles().get(0).lastIndexOf("/") + 1; - - // Append the list of input files - final List inputFiles = compactionJobInfo.inputFiles(); - // Trim the file path, leave only the SST file name without extension - inputFiles.replaceAll(s -> s.substring( - filenameOffset, s.length() - SST_FILE_EXTENSION_LENGTH)); - final String inputFilesJoined = - String.join(COMPACTION_LOG_ENTRY_FILE_DELIMITER, inputFiles); - sb.append(inputFilesJoined); - - // Insert delimiter between input files and output files - sb.append(COMPACTION_LOG_ENTRY_INPUT_OUTPUT_FILES_DELIMITER); - - // Append the list of output files - final List outputFiles = compactionJobInfo.outputFiles(); - outputFiles.replaceAll(s -> s.substring( - filenameOffset, s.length() - SST_FILE_EXTENSION_LENGTH)); - final String outputFilesJoined = - String.join(COMPACTION_LOG_ENTRY_FILE_DELIMITER, outputFiles); - sb.append(outputFilesJoined); - - // End of line + sb.append(COMPACTION_LOG_ENTRY_LINE_PREFIX_V1); + sb.append(compactionLogEntry.toEncodedString()); sb.append('\n'); - String content = sb.toString(); synchronized (this) { @@ -611,7 +594,8 @@ public void onCompactionCompleted(RocksDB db, // Populate the DAG // TODO: [SNAPSHOT] Once SnapshotChainManager is put into use, // set snapshotID to snapshotChainManager.getLatestGlobalSnapshot() - populateCompactionDAG(inputFiles, outputFiles, null, + populateCompactionDAG(compactionLogEntry.getInputFileInfoList(), + compactionLogEntry.getOutputFileInfoList(), null, db.getLatestSequenceNumber()); } } @@ -716,6 +700,7 @@ private String trimSSTFilename(String filename) { /** * Read the current Live manifest for a given RocksDB instance (Active or * Checkpoint). + * * @param rocksDB open rocksDB instance. * @return a list of SST files (without extension) in the DB. */ @@ -778,7 +763,21 @@ void processCompactionLogLine(String line) { String[] inputFiles = io[0].split(COMPACTION_LOG_ENTRY_FILE_DELIMITER); String[] outputFiles = io[1].split(COMPACTION_LOG_ENTRY_FILE_DELIMITER); - populateCompactionDAG(asList(inputFiles), asList(outputFiles), + List inputFileInfoList = Arrays.stream(inputFiles) + .map(inputFile -> new FileInfo(inputFile, null, null, null)) + .collect(Collectors.toList()); + List outputFileInfoList = Arrays.stream(outputFiles) + .map(outputFile -> new FileInfo(outputFile, null, null, null)) + .collect(Collectors.toList()); + populateCompactionDAG(inputFileInfoList, outputFileInfoList, + reconstructionLastSnapshotID, reconstructionSnapshotGeneration); + } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX_V1)) { + String io = + line.substring(COMPACTION_LOG_ENTRY_LINE_PREFIX_V1.length()); + CompactionLogEntry compactionLogEntry = + CompactionLogEntry.fromEncodedString(io); + populateCompactionDAG(compactionLogEntry.getInputFileInfoList(), + compactionLogEntry.getOutputFileInfoList(), reconstructionLastSnapshotID, reconstructionSnapshotGeneration); } else { LOG.error("Invalid line in compaction log: {}", line); @@ -864,7 +863,9 @@ private String getSSTFullPath(String sstFilenameWithoutExtension, * @param src source snapshot * @param dest destination snapshot * @param sstFilesDirForSnapDiffJob dir to create hardlinks for SST files - * for snapDiff job. + * for snapDiff job. + * @param columnFamilyToPrefixMap map containing tableName to prefix for + * the keys in the table. * @return A list of SST files without extension. * e.g. ["/path/to/sstBackupDir/000050.sst", * "/path/to/sstBackupDir/000060.sst"] @@ -872,10 +873,12 @@ private String getSSTFullPath(String sstFilenameWithoutExtension, public synchronized List getSSTDiffListWithFullPath( DifferSnapshotInfo src, DifferSnapshotInfo dest, - String sstFilesDirForSnapDiffJob + String sstFilesDirForSnapDiffJob, + Map columnFamilyToPrefixMap ) throws IOException { - List sstDiffList = getSSTDiffList(src, dest); + List sstDiffList = + getSSTDiffList(src, dest, columnFamilyToPrefixMap); return sstDiffList.stream() .map( @@ -899,24 +902,28 @@ public synchronized List getSSTDiffListWithFullPath( * * @param src source snapshot * @param dest destination snapshot + * @param columnFamilyToPrefixMap map containing tableName to prefix for + * the keys in the table. * @return A list of SST files without extension. e.g. ["000050", "000060"] */ - public synchronized List getSSTDiffList(DifferSnapshotInfo src, - DifferSnapshotInfo dest) - throws IOException { + public synchronized List getSSTDiffList( + DifferSnapshotInfo src, + DifferSnapshotInfo dest, + Map columnFamilyToPrefixMap + ) throws IOException { // TODO: Reject or swap if dest is taken after src, once snapshot chain // integration is done. - HashSet srcSnapFiles = readRocksDBLiveFiles(src.getRocksDB()); - HashSet destSnapFiles = readRocksDBLiveFiles(dest.getRocksDB()); + Set srcSnapFiles = readRocksDBLiveFiles(src.getRocksDB()); + Set destSnapFiles = readRocksDBLiveFiles(dest.getRocksDB()); - HashSet fwdDAGSameFiles = new HashSet<>(); - HashSet fwdDAGDifferentFiles = new HashSet<>(); + Set fwdDAGSameFiles = new HashSet<>(); + Set fwdDAGDifferentFiles = new HashSet<>(); LOG.debug("Doing forward diff from src '{}' to dest '{}'", src.getDbPath(), dest.getDbPath()); internalGetSSTDiffList(src, dest, srcSnapFiles, destSnapFiles, - forwardCompactionDAG, fwdDAGSameFiles, fwdDAGDifferentFiles); + fwdDAGSameFiles, fwdDAGDifferentFiles, columnFamilyToPrefixMap); if (LOG.isDebugEnabled()) { LOG.debug("Result of diff from src '" + src.getDbPath() + "' to dest '" + @@ -971,11 +978,16 @@ public void filterRelevantSstFilesFullPath(Set inputFiles, * diffing). Otherwise, add it to the differentFiles map, as it will * need further diffing. */ + @SuppressWarnings("checkstyle:ParameterNumber") synchronized void internalGetSSTDiffList( - DifferSnapshotInfo src, DifferSnapshotInfo dest, - Set srcSnapFiles, Set destSnapFiles, - MutableGraph mutableGraph, - Set sameFiles, Set differentFiles) { + DifferSnapshotInfo src, + DifferSnapshotInfo dest, + Set srcSnapFiles, + Set destSnapFiles, + Set sameFiles, + Set differentFiles, + Map columnFamilyToPrefixMap + ) { // Sanity check Preconditions.checkArgument(sameFiles.isEmpty(), "Set must be empty"); @@ -1022,7 +1034,8 @@ synchronized void internalGetSSTDiffList( final Set nextLevel = new HashSet<>(); for (CompactionNode current : currentLevel) { - LOG.debug("Processing node: {}", current.getFileName()); + LOG.debug("Processing node: '{}'", current.getFileName()); + if (current.getSnapshotGeneration() < dest.getSnapshotGeneration()) { LOG.debug("Current node's snapshot generation '{}' " + "reached destination snapshot's '{}'. " @@ -1033,7 +1046,8 @@ synchronized void internalGetSSTDiffList( continue; } - Set successors = mutableGraph.successors(current); + Set successors = + forwardCompactionDAG.successors(current); if (successors.isEmpty()) { LOG.debug("No further compaction happened to the current file. " + "Src '{}' and dest '{}' have different file: {}", @@ -1043,21 +1057,30 @@ synchronized void internalGetSSTDiffList( } for (CompactionNode node : successors) { + if (shouldSkipNode(node, columnFamilyToPrefixMap)) { + LOG.debug("Compaction node: '{}' has keys from startKey: '{}' " + + "and endKey: '{}'. columnFamilyToPrefixMap is : {}.", + node.getFileName(), node.getStartKey(), node.getEndKey(), + columnFamilyToPrefixMap); + continue; + } + if (sameFiles.contains(node.getFileName()) || differentFiles.contains(node.getFileName())) { - LOG.debug("Skipping known processed SST: {}", node.getFileName()); + LOG.debug("Skipping known processed SST: '{}'", + node.getFileName()); continue; } if (destSnapFiles.contains(node.getFileName())) { - LOG.debug("Src '{}' and dest '{}' have the same SST: {}", + LOG.debug("Src '{}' and dest '{}' have the same SST: '{}'", src.getDbPath(), dest.getDbPath(), node.getFileName()); sameFiles.add(node.getFileName()); continue; } // Queue different SST to the next level - LOG.debug("Src '{}' and dest '{}' have a different SST: {}", + LOG.debug("Src '{}' and dest '{}' have a different SST: '{}'", src.getDbPath(), dest.getDbPath(), node.getFileName()); nextLevel.add(node); } @@ -1110,7 +1133,8 @@ MutableGraph getBackwardCompactionDAG() { * @return CompactionNode */ private CompactionNode addNodeToDAG(String file, String snapshotID, - long seqNum) { + long seqNum, String startRange, + String endRange, String columnFamily) { long numKeys = 0L; try { numKeys = getSSTFileSummary(file); @@ -1119,8 +1143,8 @@ private CompactionNode addNodeToDAG(String file, String snapshotID, } catch (FileNotFoundException e) { LOG.info("Can't find SST '{}'", file); } - CompactionNode fileNode = new CompactionNode( - file, snapshotID, numKeys, seqNum); + CompactionNode fileNode = new CompactionNode(file, snapshotID, numKeys, + seqNum, startRange, endRange, columnFamily); forwardCompactionDAG.addNode(fileNode); backwardCompactionDAG.addNode(fileNode); @@ -1135,20 +1159,26 @@ private CompactionNode addNodeToDAG(String file, String snapshotID, * arbitrary String as long as it helps debugging. * @param seqNum DB transaction sequence number. */ - private void populateCompactionDAG(List inputFiles, - List outputFiles, String snapshotId, long seqNum) { + private void populateCompactionDAG(List inputFiles, + List outputFiles, + String snapshotId, + long seqNum) { if (LOG.isDebugEnabled()) { LOG.debug("Input files: {} -> Output files: {}", inputFiles, outputFiles); } - for (String outfile : outputFiles) { + for (FileInfo outfile : outputFiles) { final CompactionNode outfileNode = compactionNodeMap.computeIfAbsent( - outfile, file -> addNodeToDAG(file, snapshotId, seqNum)); + outfile.getFileName(), file -> addNodeToDAG(file, snapshotId, seqNum, + outfile.getStartKey(), outfile.getEndKey(), + outfile.getColumnFamily())); - for (String infile : inputFiles) { + for (FileInfo infile : inputFiles) { final CompactionNode infileNode = compactionNodeMap.computeIfAbsent( - infile, file -> addNodeToDAG(file, snapshotId, seqNum)); + infile.getFileName(), file -> addNodeToDAG(file, snapshotId, seqNum, + infile.getStartKey(), infile.getEndKey(), + infile.getColumnFamily())); // Draw the edges if (!outfileNode.getFileName().equals(infileNode.getFileName())) { forwardCompactionDAG.putEdge(outfileNode, infileNode); @@ -1424,7 +1454,8 @@ private List getLastCompactionSstFiles(Path compactionLogFile) { try (Stream logStream = Files.lines(compactionLogFile, UTF_8)) { logStream.forEach(logLine -> { - if (!logLine.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) { + if (!(logLine.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX) || + logLine.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX_V1))) { return; } sstFiles.set(logLine); @@ -1440,18 +1471,33 @@ private List getLastCompactionSstFiles(Path compactionLogFile) { return Collections.emptyList(); } - // Trim the beginning - lastCompactionLogEntry = lastCompactionLogEntry - .substring(COMPACTION_LOG_ENTRY_LINE_PREFIX.length()); - - String[] io = lastCompactionLogEntry - .split(COMPACTION_LOG_ENTRY_INPUT_OUTPUT_FILES_DELIMITER); - - assert (io.length == 2); - - String[] outputFiles = io[1].split(COMPACTION_LOG_ENTRY_FILE_DELIMITER); - - return asList(outputFiles); + if (lastCompactionLogEntry.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) { + // Trim the beginning + lastCompactionLogEntry = lastCompactionLogEntry + .substring(COMPACTION_LOG_ENTRY_LINE_PREFIX.length()); + + String[] io = lastCompactionLogEntry + .split(COMPACTION_LOG_ENTRY_INPUT_OUTPUT_FILES_DELIMITER); + + assert (io.length == 2); + + String[] outputFiles = io[1].split(COMPACTION_LOG_ENTRY_FILE_DELIMITER); + + return asList(outputFiles); + } else if (lastCompactionLogEntry.startsWith( + COMPACTION_LOG_ENTRY_LINE_PREFIX_V1)) { + // Trim the beginning + lastCompactionLogEntry = lastCompactionLogEntry + .substring(COMPACTION_LOG_ENTRY_LINE_PREFIX_V1.length()); + CompactionLogEntry compactionLogEntry = + CompactionLogEntry.fromEncodedString(lastCompactionLogEntry); + return compactionLogEntry.getOutputFileInfoList().stream() + .map(FileInfo::getFileName) + .collect(Collectors.toList()); + } else { + throw new IllegalStateException("Should not have reached here. " + + "Log line: " + lastCompactionLogEntry); + } } public String getSSTBackupDir() { @@ -1610,4 +1656,33 @@ public void pngPrintMutableGraph(String filePath, GraphType graphType) graph.generateImage(filePath); } + + private boolean shouldSkipNode(CompactionNode node, + Map columnFamilyToPrefixMap) { + // This is for backward compatibility. Before the compaction log V1 + // migration,startKey, endKey and columnFamily information is not persisted + // in compaction logs. + if (node.getStartKey() == null || node.getEndKey() == null || + node.getColumnFamily() == null) { + LOG.debug("Compaction node with fileName: {} doesn't have startKey, " + + "endKey and columnFamily details.", node.getFileName()); + return false; + } + + if (MapUtils.isEmpty(columnFamilyToPrefixMap)) { + LOG.debug("Provided columnFamilyToPrefixMap is null or empty."); + return false; + } + + if (!columnFamilyToPrefixMap.containsKey(node.getColumnFamily())) { + LOG.debug("SstFile: {} is for columnFamily: {} while filter map " + + "contains columnFamilies: {}.", node.getFileName(), + node.getColumnFamily(), columnFamilyToPrefixMap.keySet()); + return false; + } + + String keyPrefix = columnFamilyToPrefixMap.get(node.getColumnFamily()); + return RocksDiffUtils.isKeyWithPrefixPresent(keyPrefix, node.getStartKey(), + node.getEndKey()); + } } diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDiffUtils.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDiffUtils.java index d1acaf0c8616..9c0e6bff1980 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDiffUtils.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDiffUtils.java @@ -49,8 +49,10 @@ private RocksDiffUtils() { public static boolean isKeyWithPrefixPresent(String prefixForColumnFamily, String firstDbKey, String lastDbKey) { - return firstDbKey.compareTo(prefixForColumnFamily) <= 0 - && prefixForColumnFamily.compareTo(lastDbKey) <= 0; + String firstKeyPrefix = constructBucketKey(firstDbKey); + String endKeyPrefix = constructBucketKey(lastDbKey); + return firstKeyPrefix.compareTo(prefixForColumnFamily) <= 0 + && prefixForColumnFamily.compareTo(endKeyPrefix) <= 0; } public static String constructBucketKey(String keyName) { diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java index d7022dde89ce..b550445c35f0 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java @@ -316,13 +316,13 @@ public void testGetSSTDiffListWithoutDB(String description, try { differ.internalGetSSTDiffList( - srcSnapshot, - destSnapshot, - srcSnapshotSstFiles, - destSnapshotSstFiles, - differ.getForwardCompactionDAG(), - actualSameSstFiles, - actualDiffSstFiles); + srcSnapshot, + destSnapshot, + srcSnapshotSstFiles, + destSnapshotSstFiles, + actualSameSstFiles, + actualDiffSstFiles, + Collections.emptyMap()); } catch (RuntimeException rtEx) { if (!expectingException) { fail("Unexpected exception thrown in test."); @@ -425,7 +425,8 @@ void diffAllSnapshots(RocksDBCheckpointDiffer differ) int index = 0; for (DifferSnapshotInfo snap : snapshots) { // Returns a list of SST files to be fed into RocksDiff - List sstDiffList = differ.getSSTDiffList(src, snap); + List sstDiffList = differ.getSSTDiffList(src, snap, + Collections.emptyMap()); LOG.info("SST diff list from '{}' to '{}': {}", src.getDbPath(), snap.getDbPath(), sstDiffList); @@ -745,7 +746,10 @@ private void printMutableGraphFromAGivenNode( sstFile -> new CompactionNode(sstFile, UUID.randomUUID().toString(), 1000L, - Long.parseLong(sstFile.substring(0, 6)) + Long.parseLong(sstFile.substring(0, 6)), + null, + null, + null )) .collect(Collectors.toList())) .collect(Collectors.toList()); @@ -1259,7 +1263,6 @@ public void testSstFilePruning( waitForLock(differ, RocksDBCheckpointDiffer::pruneSstFiles); - Set actualFileSetAfterPruning; try (Stream pathStream = Files.list( Paths.get(metadataDirName + "/" + sstBackUpDirName)) diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDiffUtils.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDiffUtils.java new file mode 100644 index 000000000000..67233676f0b3 --- /dev/null +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDiffUtils.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ozone.rocksdiff; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Class to test RocksDiffUtils. + */ +public class TestRocksDiffUtils { + @Test + public void testFilterFunction() { + assertTrue(RocksDiffUtils.isKeyWithPrefixPresent( + "/vol1/bucket1/", + "/vol1/bucket1/key1", + "/vol1/bucket1/key1")); + assertTrue(RocksDiffUtils.isKeyWithPrefixPresent( + "/vol1/bucket3/", + "/vol1/bucket1/key1", + "/vol1/bucket5/key1")); + assertFalse(RocksDiffUtils.isKeyWithPrefixPresent( + "/vol1/bucket5/", + "/vol1/bucket1/key1", + "/vol1/bucket4/key9")); + assertFalse(RocksDiffUtils.isKeyWithPrefixPresent( + "/vol1/bucket2/", + "/vol1/bucket1/key1", + "/vol1/bucket1/key1")); + assertFalse(RocksDiffUtils.isKeyWithPrefixPresent( + "/vol1/bucket/", + "/vol1/bucket1/key1", + "/vol1/bucket1/key1")); + assertTrue(RocksDiffUtils.isKeyWithPrefixPresent( + "/volume/bucket/", + "/volume/bucket/key-1", + "/volume/bucket2/key-97")); + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java index d21fdb39584e..45ef28c76556 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java @@ -60,6 +60,7 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.time.Duration; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -252,7 +253,8 @@ public void testDAGReconstruction() final File checkpointSnap2 = new File(snap2.getDbPath()); GenericTestUtils.waitFor(checkpointSnap2::exists, 2000, 20000); - List sstDiffList21 = differ.getSSTDiffList(snap2, snap1); + List sstDiffList21 = differ.getSSTDiffList(snap2, snap1, + Collections.emptyMap()); LOG.debug("Got diff list: {}", sstDiffList21); // Delete 1000 keys, take a 3rd snapshot, and do another diff @@ -273,13 +275,16 @@ public void testDAGReconstruction() final File checkpointSnap3 = new File(snap3.getDbPath()); GenericTestUtils.waitFor(checkpointSnap3::exists, 2000, 20000); - List sstDiffList32 = differ.getSSTDiffList(snap3, snap2); + List sstDiffList32 = differ.getSSTDiffList(snap3, snap2, + Collections.emptyMap()); // snap3-snap1 diff result is a combination of snap3-snap2 and snap2-snap1 - List sstDiffList31 = differ.getSSTDiffList(snap3, snap1); + List sstDiffList31 = differ.getSSTDiffList(snap3, snap1, + Collections.emptyMap()); // Same snapshot. Result should be empty list - List sstDiffList22 = differ.getSSTDiffList(snap2, snap2); + List sstDiffList22 = differ.getSSTDiffList(snap2, snap2, + Collections.emptyMap()); Assertions.assertTrue(sstDiffList22.isEmpty()); snapDB1.close(); snapDB2.close(); @@ -308,13 +313,16 @@ public void testDAGReconstruction() volumeName, bucketName, "snap3", ((RDBStore)((OmSnapshot)snapDB3.get()) .getMetadataManager().getStore()).getDb().getManagedRocksDb()); - List sstDiffList21Run2 = differ.getSSTDiffList(snap2, snap1); + List sstDiffList21Run2 = differ.getSSTDiffList(snap2, snap1, + Collections.emptyMap()); Assertions.assertEquals(sstDiffList21, sstDiffList21Run2); - List sstDiffList32Run2 = differ.getSSTDiffList(snap3, snap2); + List sstDiffList32Run2 = differ.getSSTDiffList(snap3, snap2, + Collections.emptyMap()); Assertions.assertEquals(sstDiffList32, sstDiffList32Run2); - List sstDiffList31Run2 = differ.getSSTDiffList(snap3, snap1); + List sstDiffList31Run2 = differ.getSSTDiffList(snap3, snap1, + Collections.emptyMap()); Assertions.assertEquals(sstDiffList31, sstDiffList31Run2); snapDB1.close(); snapDB2.close(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java index 0b6b17ea64bd..1982e1b327e6 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java @@ -20,13 +20,11 @@ import com.google.common.annotations.VisibleForTesting; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.utils.BackgroundService; import org.apache.hadoop.hdds.utils.BackgroundTask; import org.apache.hadoop.hdds.utils.BackgroundTaskQueue; import org.apache.hadoop.hdds.utils.BackgroundTaskResult; -import org.apache.hadoop.hdds.utils.BooleanTriFunction; import org.apache.hadoop.hdds.utils.db.RDBStore; import org.apache.hadoop.hdds.utils.db.RocksDatabase; import org.apache.hadoop.hdds.utils.db.Table; @@ -36,23 +34,21 @@ import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; -import org.apache.ozone.rocksdiff.RocksDiffUtils; import org.rocksdb.RocksDBException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_SST_DELETING_LIMIT_PER_TASK; import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_SST_DELETING_LIMIT_PER_TASK_DEFAULT; import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK; +import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.getColumnFamilyToPrefixMap; /** * When snapshots are taken, an entire snapshot of the @@ -83,18 +79,6 @@ public class SstFilteringService extends BackgroundService private AtomicBoolean running; - // Note: This filter only works till snapshots are readable only. - // In the future, if snapshots are changed to writable as well, - // this will need to be revisited. - static final BooleanTriFunction - FILTER_FUNCTION = - (first, last, prefix) -> { - String firstBucketKey = RocksDiffUtils.constructBucketKey(first); - String lastBucketKey = RocksDiffUtils.constructBucketKey(last); - return RocksDiffUtils - .isKeyWithPrefixPresent(prefix, firstBucketKey, lastBucketKey); - }; - public SstFilteringService(long interval, TimeUnit unit, long serviceTimeout, OzoneManager ozoneManager, OzoneConfiguration configuration) { super("SstFilteringService", interval, unit, SST_FILTERING_CORE_POOL_SIZE, @@ -192,8 +176,10 @@ public BackgroundTaskResult call() throws Exception { LOG.debug("Processing snapshot {} to filter relevant SST Files", snapShotTableKey); - List> prefixPairs = constructPrefixPairs( - snapshotInfo); + Map prefixPairs = getColumnFamilyToPrefixMap( + ozoneManager.getMetadataManager(), + snapshotInfo.getVolumeName(), + snapshotInfo.getBucketName()); try ( ReferenceCounted @@ -205,7 +191,7 @@ public BackgroundTaskResult call() throws Exception { RocksDatabase db = rdbStore.getDb(); try (BootstrapStateHandler.Lock lock = getBootstrapStateLock() .lock()) { - db.deleteFilesNotMatchingPrefix(prefixPairs, FILTER_FUNCTION); + db.deleteFilesNotMatchingPrefix(prefixPairs); } } catch (OMException ome) { // FILE_NOT_FOUND is obtained when the snapshot is deleted @@ -238,43 +224,8 @@ public BackgroundTaskResult call() throws Exception { // nothing to return here return BackgroundTaskResult.EmptyTaskResult.newResult(); } - - /** - * @param snapshotInfo - * @return a list of pairs (tableName,keyPrefix). - * @throws IOException - */ - private List> constructPrefixPairs( - SnapshotInfo snapshotInfo) throws IOException { - String volumeName = snapshotInfo.getVolumeName(); - String bucketName = snapshotInfo.getBucketName(); - - long volumeId = ozoneManager.getMetadataManager().getVolumeId(volumeName); - // TODO : HDDS-6984 buckets can be deleted via ofs - // handle deletion of bucket case. - long bucketId = - ozoneManager.getMetadataManager().getBucketId(volumeName, bucketName); - - String filterPrefix = - OM_KEY_PREFIX + volumeName + OM_KEY_PREFIX + bucketName - + OM_KEY_PREFIX; - - String filterPrefixFSO = - OM_KEY_PREFIX + volumeId + OM_KEY_PREFIX + bucketId - + OM_KEY_PREFIX; - - List> prefixPairs = new ArrayList<>(); - prefixPairs - .add(Pair.of(OmMetadataManagerImpl.KEY_TABLE, filterPrefix)); - prefixPairs.add( - Pair.of(OmMetadataManagerImpl.DIRECTORY_TABLE, filterPrefixFSO)); - prefixPairs - .add(Pair.of(OmMetadataManagerImpl.FILE_TABLE, filterPrefixFSO)); - return prefixPairs; - } } - @Override public BackgroundTaskQueue getTasks() { BackgroundTaskQueue queue = new BackgroundTaskQueue(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java index 9067873ce9ce..72dfc0cc1e7d 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java @@ -121,6 +121,7 @@ import static org.apache.hadoop.ozone.om.helpers.SnapshotInfo.getTableKey; import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.checkSnapshotActive; import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.dropColumnFamilyHandle; +import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.getColumnFamilyToPrefixMap; import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.getSnapshotInfo; import static org.apache.hadoop.ozone.snapshot.CancelSnapshotDiffResponse.CancelMessage.CANCEL_FAILED; import static org.apache.hadoop.ozone.snapshot.CancelSnapshotDiffResponse.CancelMessage.CANCEL_ALREADY_CANCELLED_JOB; @@ -1212,8 +1213,11 @@ Set getDeltaFiles(OmSnapshot fromSnapshot, LOG.debug("Calling RocksDBCheckpointDiffer"); try { - List sstDiffList = - differ.getSSTDiffListWithFullPath(toDSI, fromDSI, diffDir); + Map columnFamilyToPrefixMap = + getColumnFamilyToPrefixMap(ozoneManager.getMetadataManager(), + volume, bucket); + List sstDiffList = differ.getSSTDiffListWithFullPath(toDSI, + fromDSI, diffDir, columnFamilyToPrefixMap); deltaFiles.addAll(sstDiffList); } catch (Exception exception) { LOG.warn("Failed to get SST diff file using RocksDBCheckpointDiffer. " + diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java index ef7e4e895e7a..9271195c2338 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java @@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.om.snapshot; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB; +import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.SnapshotChainManager; @@ -32,8 +33,14 @@ import java.io.File; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; import java.util.UUID; +import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE; +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE; +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TIMEOUT; @@ -155,4 +162,30 @@ public static SnapshotInfo getNextActiveSnapshot(SnapshotInfo snapInfo, } return null; } + + /** + * Return a map column family to prefix for the keys in the table for + * the given volume and bucket. + * Column families, map is returned for, are keyTable, dirTable and fileTable. + */ + public static Map getColumnFamilyToPrefixMap( + OMMetadataManager omMetadataManager, + String volumeName, + String bucketName + ) throws IOException { + long volumeId = omMetadataManager.getVolumeId(volumeName); + long bucketId = omMetadataManager.getBucketId(volumeName, bucketName); + + String keyPrefix = OM_KEY_PREFIX + volumeName + OM_KEY_PREFIX + bucketName + + OM_KEY_PREFIX; + + String keyPrefixFso = OM_KEY_PREFIX + volumeId + OM_KEY_PREFIX + + bucketId + OM_KEY_PREFIX; + + Map comlumnFamilyToPrefixMap = new HashMap<>(); + comlumnFamilyToPrefixMap.put(KEY_TABLE, keyPrefix); + comlumnFamilyToPrefixMap.put(DIRECTORY_TABLE, keyPrefixFso); + comlumnFamilyToPrefixMap.put(FILE_TABLE, keyPrefixFso); + return comlumnFamilyToPrefixMap; + } } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSstFilteringService.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSstFilteringService.java index a8c126f43bd8..31371c81d044 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSstFilteringService.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSstFilteringService.java @@ -372,30 +372,6 @@ private void createKey(OzoneManagerProtocol managerProtocol, managerProtocol.commitKey(keyArg, session.getId()); } - @Test - public void testFilterFunction() { - assertTrue(SstFilteringService.FILTER_FUNCTION.apply( - "/vol1/bucket1/key1", - "/vol1/bucket1/key1", - "/vol1/bucket1/")); - assertTrue(SstFilteringService.FILTER_FUNCTION.apply( - "/vol1/bucket1/key1", - "/vol1/bucket5/key1", - "/vol1/bucket3/")); - assertFalse(SstFilteringService.FILTER_FUNCTION.apply( - "/vol1/bucket1/key1", - "/vol1/bucket4/key9", - "/vol1/bucket5/")); - assertFalse(SstFilteringService.FILTER_FUNCTION.apply( - "/vol1/bucket1/key1", - "/vol1/bucket1/key1", - "/vol1/bucket2/")); - assertFalse(SstFilteringService.FILTER_FUNCTION.apply( - "/vol1/bucket1/key1", - "/vol1/bucket1/key1", - "/vol1/bucket/")); - } - /** * Test to verify the data integrity after SST filtering service runs. * This test creates 150 keys randomly in one of the three buckets. It also diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java index 7a07f97f5669..1cd0fbcbbf3c 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java @@ -430,7 +430,8 @@ public void testGetDeltaFilesWithDag(int numberOfFiles) throws IOException { when(differ.getSSTDiffListWithFullPath( any(DifferSnapshotInfo.class), any(DifferSnapshotInfo.class), - eq(diffDir)) + eq(diffDir), + anyMap()) ).thenReturn(Lists.newArrayList(randomStrings)); ReferenceCounted rcFromSnapshot = @@ -495,7 +496,8 @@ public void testGetDeltaFilesWithFullDiff(int numberOfFiles, when(differ.getSSTDiffListWithFullPath( any(DifferSnapshotInfo.class), any(DifferSnapshotInfo.class), - anyString())) + anyString(), + anyMap())) .thenReturn(Collections.emptyList()); } @@ -562,7 +564,8 @@ public void testGetDeltaFilesWithDifferThrowException(int numberOfFiles) .getSSTDiffListWithFullPath( any(DifferSnapshotInfo.class), any(DifferSnapshotInfo.class), - anyString()); + anyString(), + anyMap()); ReferenceCounted rcFromSnapshot = snapshotCache.get(snap1.toString()); From 8db7add00a75b0a3277543f3db891407f575a40e Mon Sep 17 00:00:00 2001 From: Hemant Kumar Date: Wed, 13 Sep 2023 15:42:12 -0700 Subject: [PATCH 2/6] Use rockDB column family instead of text file for compaction logs --- .../org/apache/hadoop/ozone/OzoneConsts.java | 5 + .../apache/hadoop/hdds/utils/db/RDBStore.java | 9 + .../hadoop/hdds/utils/db/RocksDatabase.java | 10 +- .../src/main/proto/hdds.proto | 12 +- ...{FileInfo.java => CompactionFileInfo.java} | 19 +- .../ozone/rocksdiff/CompactionLogEntry.java | 189 +++++----- .../rocksdiff/RocksDBCheckpointDiffer.java | 325 ++++++++++-------- .../ozone/om/OmMetadataManagerImpl.java | 6 +- .../hadoop/ozone/om/SstFilteringService.java | 10 +- 9 files changed, 345 insertions(+), 240 deletions(-) rename hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/{FileInfo.java => CompactionFileInfo.java} (78%) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java index 27855d187d32..e802de066692 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java @@ -606,4 +606,9 @@ private OzoneConsts() { */ public static final String SNAPSHOT_INFO_TABLE = "snapshotInfoTable"; + /** + * DB compaction log table name. Referenced in RDBStore. + */ + public static final String COMPACTION_LOG_TABLE = + "compactionLogTable"; } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java index 7ccb01d79f37..b8536bc8511a 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java @@ -49,6 +49,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.ozone.OzoneConsts.COMPACTION_LOG_TABLE; import static org.apache.hadoop.ozone.OzoneConsts.OM_CHECKPOINT_DIR; import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_CHECKPOINT_DIR; @@ -154,6 +155,14 @@ public RDBStore(File dbFile, ManagedDBOptions dbOptions, // sequence number as current compaction log filename. rocksDBCheckpointDiffer.setCurrentCompactionLog( db.getLatestSequenceNumber()); + + ColumnFamily compactionLogTableTableCF = + db.getColumnFamily(COMPACTION_LOG_TABLE); + Preconditions.checkNotNull(compactionLogTableTableCF, + "CompactionLogTable column family handle should not be null"); + rocksDBCheckpointDiffer.setCompactionLogTableCFHandle( + compactionLogTableTableCF.getHandle()); + rocksDBCheckpointDiffer.setActiveRocksDB(db.getManagedRocksDb().get()); // Load all previous compaction logs rocksDBCheckpointDiffer.loadAllCompactionLogs(); } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java index f49bd0768887..dc348c1cc1c7 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java @@ -979,10 +979,14 @@ public void deleteFilesNotMatchingPrefix(Map prefixPairs) boolean isKeyWithPrefixPresent = RocksDiffUtils.isKeyWithPrefixPresent( prefixForColumnFamily, firstDbKey, lastDbKey); if (!isKeyWithPrefixPresent) { - LOG.info("Deleting sst file {} corresponding to column family" - + " {} from db: {}", liveFileMetaData.fileName(), + LOG.info("Deleting sst file: {} with start key: {} and end key: {} " + + "corresponding to column family {} from db: {}. " + + "Prefix for the column family: {}.", + liveFileMetaData.fileName(), + firstDbKey, lastDbKey, StringUtils.bytes2String(liveFileMetaData.columnFamilyName()), - db.get().getName()); + db.get().getName(), + prefixForColumnFamily); db.deleteFile(liveFileMetaData); } } diff --git a/hadoop-hdds/interface-client/src/main/proto/hdds.proto b/hadoop-hdds/interface-client/src/main/proto/hdds.proto index 8090c62d8248..ee35943bfffa 100644 --- a/hadoop-hdds/interface-client/src/main/proto/hdds.proto +++ b/hadoop-hdds/interface-client/src/main/proto/hdds.proto @@ -481,15 +481,17 @@ message DeletedBlocksTransactionInfo { optional int32 count = 4; } -message FileInfoProto { - optional string fileName = 1; +message CompactionFileInfoProto { + required string fileName = 1; optional string startKey = 2; optional string endKey = 3; optional string columnFamily = 4; } message CompactionLogEntryProto { - optional uint64 dbSequenceNumber = 1; - repeated FileInfoProto inputFileIntoList = 2; - repeated FileInfoProto outputFileIntoList = 3; + required uint64 dbSequenceNumber = 1; + required uint64 compactionTime = 2; + repeated CompactionFileInfoProto inputFileIntoList = 3; + repeated CompactionFileInfoProto outputFileIntoList = 4; + optional string compactionReason = 5; } diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/FileInfo.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionFileInfo.java similarity index 78% rename from hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/FileInfo.java rename to hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionFileInfo.java index bfa65534955c..8083ff12a115 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/FileInfo.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionFileInfo.java @@ -23,16 +23,16 @@ /** * Dao to keep SST file information in the compaction log. */ -public class FileInfo { +public class CompactionFileInfo { private final String fileName; private final String startKey; private final String endKey; private final String columnFamily; - public FileInfo(String fileName, - String startRange, - String endRange, - String columnFamily) { + public CompactionFileInfo(String fileName, + String startRange, + String endRange, + String columnFamily) { this.fileName = fileName; this.startKey = startRange; this.endKey = endRange; @@ -55,8 +55,8 @@ public String getColumnFamily() { return columnFamily; } - public HddsProtos.FileInfoProto getProtobuf() { - return HddsProtos.FileInfoProto.newBuilder() + public HddsProtos.CompactionFileInfoProto getProtobuf() { + return HddsProtos.CompactionFileInfoProto.newBuilder() .setFileName(fileName) .setStartKey(startKey) .setEndKey(endKey) @@ -64,8 +64,9 @@ public HddsProtos.FileInfoProto getProtobuf() { .build(); } - public static FileInfo getFromProtobuf(HddsProtos.FileInfoProto proto) { - return new FileInfo(proto.getFileName(), proto.getStartKey(), + public static CompactionFileInfo getFromProtobuf( + HddsProtos.CompactionFileInfoProto proto) { + return new CompactionFileInfo(proto.getFileName(), proto.getStartKey(), proto.getEndKey(), proto.getColumnFamily()); } diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java index 85f61bd4df7c..106f928d835f 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java @@ -18,7 +18,6 @@ package org.apache.ozone.rocksdiff; -import com.google.protobuf.InvalidProtocolBufferException; import org.apache.commons.collections.CollectionUtils; import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.CompactionLogEntryProto; @@ -29,7 +28,6 @@ import org.rocksdb.SstFileReaderIterator; import java.util.ArrayList; -import java.util.Base64; import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -41,22 +39,28 @@ */ public class CompactionLogEntry { private final long dbSequenceNumber; - private final List inputFileInfoList; - private final List outputFileInfoList; + private final long compactionTime; + private final List inputFileInfoList; + private final List outputFileInfoList; + private final String compactionReason; public CompactionLogEntry(long dbSequenceNumber, - List inputFileInfoList, - List outputFileInfoList) { + long compactionTime, + List inputFileInfoList, + List outputFileInfoList, + String compactionReason) { this.dbSequenceNumber = dbSequenceNumber; + this.compactionTime = compactionTime; this.inputFileInfoList = inputFileInfoList; this.outputFileInfoList = outputFileInfoList; + this.compactionReason = compactionReason; } - public List getInputFileInfoList() { + public List getInputFileInfoList() { return inputFileInfoList; } - public List getOutputFileInfoList() { + public List getOutputFileInfoList() { return outputFileInfoList; } @@ -64,10 +68,19 @@ public long getDbSequenceNumber() { return dbSequenceNumber; } + public long getCompactionTime() { + return compactionTime; + } + public CompactionLogEntryProto getProtobuf() { CompactionLogEntryProto.Builder builder = CompactionLogEntryProto .newBuilder() - .setDbSequenceNumber(dbSequenceNumber); + .setDbSequenceNumber(dbSequenceNumber) + .setCompactionTime(compactionTime); + + if (compactionReason != null) { + builder.setCompactionReason(compactionReason); + } if (inputFileInfoList != null) { inputFileInfoList.forEach(fileInfo -> @@ -84,36 +97,19 @@ public CompactionLogEntryProto getProtobuf() { public static CompactionLogEntry getFromProtobuf( CompactionLogEntryProto proto) { - List inputFileInfo = proto.getInputFileIntoListList().stream() - .map(FileInfo::getFromProtobuf) + List inputFileInfo = proto.getInputFileIntoListList() + .stream() + .map(CompactionFileInfo::getFromProtobuf) .collect(Collectors.toList()); - List outputFileInfo = proto.getOutputFileIntoListList().stream() - .map(FileInfo::getFromProtobuf) + List outputFileInfo = proto.getOutputFileIntoListList() + .stream() + .map(CompactionFileInfo::getFromProtobuf) .collect(Collectors.toList()); - return new CompactionLogEntry(proto.getDbSequenceNumber(), - inputFileInfo, outputFileInfo); - } - - public String toEncodedString() { - // Encoding is used to deal with \n. Protobuf appends \n after each - // parameter. If ByteArray is simply converted to a string or - // protobuf.toString(), it will contain \n and will be added to the log. - // Which creates a problem while reading compaction logs. - // Compaction log lines are read sequentially assuming each line is one - // compaction log entry. - return Base64.getEncoder().encodeToString(getProtobuf().toByteArray()); - } - - public static CompactionLogEntry fromEncodedString(String string) { - try { - byte[] decode = Base64.getDecoder().decode(string); - return getFromProtobuf(CompactionLogEntryProto.parseFrom(decode)); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } + proto.getCompactionTime(), inputFileInfo, outputFileInfo, + proto.getCompactionReason()); } @Override @@ -123,57 +119,90 @@ public String toString() { outputFileInfoList); } - public static CompactionLogEntry fromCompactionFiles( - long dbSequenceNumber, - List inputFiles, - List outputFiles - ) { - - try (ManagedOptions options = new ManagedOptions(); - ManagedReadOptions readOptions = new ManagedReadOptions()) { - List inputFileInfos = convertFileInfo(inputFiles, options, - readOptions); - List outputFileInfos = convertFileInfo(outputFiles, options, - readOptions); - return new CompactionLogEntry(dbSequenceNumber, inputFileInfos, - outputFileInfos); + + /** + * Builder of CompactionLogEntry. + */ + public static class Builder { + private long dbSequenceNumber; + private long compactionTime; + private List inputFiles; + private List outputFiles; + private String compactionReason; + + public Builder() { } - } - private static List convertFileInfo( - List sstFiles, - ManagedOptions options, - ManagedReadOptions readOptions - ) { - if (CollectionUtils.isEmpty(sstFiles)) { - return Collections.emptyList(); + public Builder setDbSequenceNumber(long dbSequenceNumber) { + this.dbSequenceNumber = dbSequenceNumber; + return this; } - final int fileNameOffset = sstFiles.get(0).lastIndexOf("/") + 1; - List response = new ArrayList<>(); - - for (String sstFile : sstFiles) { - String fileName = sstFile.substring(fileNameOffset, - sstFile.length() - SST_FILE_EXTENSION_LENGTH); - SstFileReader fileReader = new SstFileReader(options); - try { - 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()); - - FileInfo fileInfo = new FileInfo(fileName, startKey, endKey, - columnFamily); - response.add(fileInfo); - } catch (RocksDBException rocksDBException) { - throw new RuntimeException("Failed to read SST file: " + sstFile, - rocksDBException); + public Builder setCompactionTime(long compactionTime) { + this.compactionTime = compactionTime; + return this; + } + public Builder setInputFiles(List inputFiles) { + this.inputFiles = inputFiles; + return this; + } + + public Builder setOutputFiles(List outputFiles) { + this.outputFiles = outputFiles; + return this; + } + + public Builder setCompactionReason(String compactionReason) { + this.compactionReason = compactionReason; + return this; + } + + public CompactionLogEntry build() { + try (ManagedOptions options = new ManagedOptions(); + ManagedReadOptions readOptions = new ManagedReadOptions()) { + return new CompactionLogEntry(dbSequenceNumber, compactionTime, + toFileInfoList(inputFiles, options, readOptions), + toFileInfoList(outputFiles, options, readOptions), + compactionReason); + } + } + + private List toFileInfoList( + List sstFiles, + ManagedOptions options, + ManagedReadOptions readOptions + ) { + if (CollectionUtils.isEmpty(sstFiles)) { + return Collections.emptyList(); + } + + final int fileNameOffset = sstFiles.get(0).lastIndexOf("/") + 1; + List response = new ArrayList<>(); + + for (String sstFile : sstFiles) { + String fileName = sstFile.substring(fileNameOffset, + sstFile.length() - SST_FILE_EXTENSION_LENGTH); + SstFileReader fileReader = new SstFileReader(options); + try { + 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()); + + CompactionFileInfo + fileInfo = new CompactionFileInfo(fileName, startKey, endKey, + columnFamily); + response.add(fileInfo); + } catch (RocksDBException rocksDBException) { + throw new RuntimeException("Failed to read SST file: " + sstFile, + rocksDBException); + } } + return response; } - return response; } } diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java index b1a322597e8c..3f0e2bb5281d 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java @@ -32,10 +32,13 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import com.google.protobuf.InvalidProtocolBufferException; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.CompactionLogEntryProto; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.hdds.utils.Scheduler; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; @@ -115,7 +118,6 @@ public class RocksDBCheckpointDiffer implements AutoCloseable, private final String metadataDir; private final String sstBackupDir; private final String activeDBLocationStr; - private final String compactionLogDir; /** @@ -140,11 +142,6 @@ public class RocksDBCheckpointDiffer implements AutoCloseable, */ private static final String COMPACTION_LOG_ENTRY_LINE_PREFIX = "C "; - /** - * Marks the beginning of a compaction log entry. - */ - private static final String COMPACTION_LOG_ENTRY_LINE_PREFIX_V1 = "C1 "; - /** * Prefix for the sequence number line when writing to compaction log * right after taking an Ozone snapshot. @@ -181,6 +178,8 @@ public class RocksDBCheckpointDiffer implements AutoCloseable, * Used during DAG reconstruction. */ private long reconstructionSnapshotGeneration; + private long reconstructionSnapshotCreationTime; + private String reconstructionCompactionReason; private String reconstructionLastSnapshotID; private final Scheduler scheduler; @@ -194,6 +193,9 @@ public class RocksDBCheckpointDiffer implements AutoCloseable, private final String dagPruningServiceName = "CompactionDagPruningService"; private AtomicBoolean suspended; + private ColumnFamilyHandle compactionLogTableCFHandle; + private RocksDB activeRocksDB; + /** * This is a package private constructor and should not be used other than * testing. Caller should use RocksDBCheckpointDifferHolder#getInstance() to @@ -464,6 +466,26 @@ public void setSnapshotInfoTableCFHandle( this.snapshotInfoTableCFHandle = snapshotInfoTableCFHandle; } + /** + * Set CompactionLogTable DB column family handle to be used in DB listener. + * @param compactionLogTableCFHandle ColumnFamilyHandle + */ + public void setCompactionLogTableCFHandle( + ColumnFamilyHandle compactionLogTableCFHandle) { + Preconditions.checkNotNull(compactionLogTableCFHandle, + "Column family handle should not be null"); + this.compactionLogTableCFHandle = compactionLogTableCFHandle; + } + + /** + * Set activeRocksDB to access CompactionLogTable. + * @param activeRocksDB RocksDB + */ + public void setActiveRocksDB(RocksDB activeRocksDB) { + Preconditions.checkNotNull(activeRocksDB, "RocksDB should not be null."); + this.activeRocksDB = activeRocksDB; + } + /** * Helper method to check whether the SnapshotInfoTable column family is empty * in a given DB instance. @@ -555,25 +577,19 @@ public void onCompactionCompleted(RocksDB db, return; } - CompactionLogEntry compactionLogEntry = CompactionLogEntry - .fromCompactionFiles(db.getLatestSequenceNumber(), - compactionJobInfo.inputFiles(), - compactionJobInfo.outputFiles()); - - final StringBuilder sb = new StringBuilder(); + long trxId = db.getLatestSequenceNumber(); + CompactionLogEntry.Builder builder = new CompactionLogEntry.Builder() + .setDbSequenceNumber(trxId) + .setCompactionTime(System.currentTimeMillis()) + .setInputFiles(compactionJobInfo.inputFiles()) + .setOutputFiles(compactionJobInfo.outputFiles()); if (LOG.isDebugEnabled()) { - // Print compaction reason for this entry in the log file - // e.g. kLevelL0FilesNum / kLevelMaxLevelSize. - sb.append(COMPACTION_LOG_COMMENT_LINE_PREFIX) - .append(compactionJobInfo.compactionReason()) - .append('\n'); + builder = builder.setCompactionReason( + compactionJobInfo.compactionReason().toString()); } - sb.append(COMPACTION_LOG_ENTRY_LINE_PREFIX_V1); - sb.append(compactionLogEntry.toEncodedString()); - sb.append('\n'); - String content = sb.toString(); + CompactionLogEntry compactionLogEntry = builder.build(); synchronized (this) { if (closed) { @@ -588,8 +604,8 @@ public void onCompactionCompleted(RocksDB db, waitForTarballCreation(); - // Write input and output file names to compaction log - appendToCurrentCompactionLog(content); + // Add the compaction log entry to Compaction log table. + addToCompactionLogTable(compactionLogEntry); // Populate the DAG // TODO: [SNAPSHOT] Once SnapshotChainManager is put into use, @@ -602,6 +618,29 @@ public void onCompactionCompleted(RocksDB db, }; } + private void addToCompactionLogTable(CompactionLogEntry compactionLogEntry) { + // This is temporary till tests are fixed. + if (compactionLogTableCFHandle == null) { + return; + } + + // Key in the transactionId-currentTime + // Just trxId can't be used because multiple compaction might be + // running, and it is possible no new entry was added to DB. + // Adding current time to transactionId eliminates key collision. + String keyString = compactionLogEntry.getDbSequenceNumber() + "-" + + compactionLogEntry.getCompactionTime(); + + byte[] key = keyString.getBytes(UTF_8); + byte[] value = compactionLogEntry.getProtobuf().toByteArray(); + try { + activeRocksDB.put(compactionLogTableCFHandle, key, value); + } catch (RocksDBException exception) { + // TODO: Revisit exception handling before merging the PR. + throw new RuntimeException(exception); + } + } + /** * Check if there is any in_progress tarball creation request and wait till * all tarball creation finish, and it gets notified. @@ -725,20 +764,24 @@ public HashSet readRocksDBLiveFiles(ManagedRocksDB rocksDB) { } /** - * Process each line of compaction log text file input and populate the DAG. + * Process log line of compaction log text file input and populate the DAG. + * It also adds the compaction log entry to compaction log table. */ void processCompactionLogLine(String line) { LOG.debug("Processing line: {}", line); synchronized (this) { - if (line.startsWith("#")) { + if (line.startsWith(COMPACTION_LOG_COMMENT_LINE_PREFIX)) { // Skip comments LOG.debug("Comment line, skipped"); + reconstructionCompactionReason = + line.substring(COMPACTION_LOG_COMMENT_LINE_PREFIX.length()); } else if (line.startsWith(COMPACTION_LOG_SEQ_NUM_LINE_PREFIX)) { SnapshotLogInfo snapshotLogInfo = getSnapshotLogInfo(line); reconstructionSnapshotGeneration = snapshotLogInfo.snapshotGenerationId; reconstructionLastSnapshotID = snapshotLogInfo.snapshotId; + reconstructionSnapshotCreationTime = snapshotLogInfo.snapshotCreatedAt; } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) { // Compaction log entry is like following: // C sequence_number input_files:output_files @@ -763,22 +806,9 @@ void processCompactionLogLine(String line) { String[] inputFiles = io[0].split(COMPACTION_LOG_ENTRY_FILE_DELIMITER); String[] outputFiles = io[1].split(COMPACTION_LOG_ENTRY_FILE_DELIMITER); - List inputFileInfoList = Arrays.stream(inputFiles) - .map(inputFile -> new FileInfo(inputFile, null, null, null)) - .collect(Collectors.toList()); - List outputFileInfoList = Arrays.stream(outputFiles) - .map(outputFile -> new FileInfo(outputFile, null, null, null)) - .collect(Collectors.toList()); - populateCompactionDAG(inputFileInfoList, outputFileInfoList, - reconstructionLastSnapshotID, reconstructionSnapshotGeneration); - } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX_V1)) { - String io = - line.substring(COMPACTION_LOG_ENTRY_LINE_PREFIX_V1.length()); - CompactionLogEntry compactionLogEntry = - CompactionLogEntry.fromEncodedString(io); - populateCompactionDAG(compactionLogEntry.getInputFileInfoList(), - compactionLogEntry.getOutputFileInfoList(), - reconstructionLastSnapshotID, reconstructionSnapshotGeneration); + addFileInfoToCompactionLogTable(reconstructionSnapshotGeneration, + reconstructionSnapshotCreationTime, inputFiles, outputFiles, + reconstructionCompactionReason); } else { LOG.error("Invalid line in compaction log: {}", line); } @@ -786,9 +816,10 @@ void processCompactionLogLine(String line) { } /** - * Helper to read compaction log to the internal DAG. + * Helper to read compaction log file to the internal DAG and compaction log + * table. */ - private void readCompactionLogToDAG(String currCompactionLogPath) { + private void readCompactionLogFile(String currCompactionLogPath) { LOG.debug("Loading compaction log: {}", currCompactionLogPath); try (Stream logLineStream = Files.lines(Paths.get(currCompactionLogPath), UTF_8)) { @@ -802,20 +833,25 @@ private void readCompactionLogToDAG(String currCompactionLogPath) { * Load existing compaction log files to the in-memory DAG. * This only needs to be done once during OM startup. */ - public void loadAllCompactionLogs() { + public void addEntriesFromLogFilesToDagAndCompactionLogTable() { synchronized (this) { if (compactionLogDir == null) { throw new RuntimeException("Compaction log directory must be set " + "first"); } reconstructionSnapshotGeneration = 0L; + reconstructionSnapshotCreationTime = 0L; + reconstructionCompactionReason = null; try { try (Stream pathStream = Files.list(Paths.get(compactionLogDir)) .filter(e -> e.toString().toLowerCase() .endsWith(COMPACTION_LOG_FILE_NAME_SUFFIX)) .sorted()) { for (Path logPath : pathStream.collect(Collectors.toList())) { - readCompactionLogToDAG(logPath.toString()); + readCompactionLogFile(logPath.toString()); + // Delete the file once entries are added to compaction table + // so that on next restart, only compaction log table is used. + Files.deleteIfExists(logPath); } } } catch (IOException e) { @@ -825,6 +861,29 @@ public void loadAllCompactionLogs() { } } + public void loadAllCompactionLogs() { + synchronized (this) { + addEntriesFromLogFilesToDagAndCompactionLogTable(); + try (ManagedRocksIterator managedRocksIterator = new ManagedRocksIterator( + activeRocksDB.newIterator(compactionLogTableCFHandle))) { + managedRocksIterator.get().seekToFirst(); + while (managedRocksIterator.get().isValid()) { + byte[] value = managedRocksIterator.get().value(); + CompactionLogEntry compactionLogEntry = + CompactionLogEntry.getFromProtobuf( + CompactionLogEntryProto.parseFrom(value)); + populateCompactionDAG(compactionLogEntry.getInputFileInfoList(), + compactionLogEntry.getOutputFileInfoList(), + null, + compactionLogEntry.getDbSequenceNumber()); + managedRocksIterator.get().next(); + } + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + } + } + /** * Helper function that prepends SST file name with SST backup directory path * (or DB checkpoint path if compaction hasn't happened yet as SST files won't @@ -1056,33 +1115,33 @@ synchronized void internalGetSSTDiffList( continue; } - for (CompactionNode node : successors) { - if (shouldSkipNode(node, columnFamilyToPrefixMap)) { - LOG.debug("Compaction node: '{}' has keys from startKey: '{}' " + + for (CompactionNode currentNode : successors) { + if (shouldSkipNode(currentNode, columnFamilyToPrefixMap)) { + LOG.debug("CurrentNode: '{}' has keys from startKey: '{}' " + "and endKey: '{}'. columnFamilyToPrefixMap is : {}.", - node.getFileName(), node.getStartKey(), node.getEndKey(), - columnFamilyToPrefixMap); + currentNode.getFileName(), currentNode.getStartKey(), + currentNode.getEndKey(), columnFamilyToPrefixMap); continue; } - if (sameFiles.contains(node.getFileName()) || - differentFiles.contains(node.getFileName())) { + if (sameFiles.contains(currentNode.getFileName()) || + differentFiles.contains(currentNode.getFileName())) { LOG.debug("Skipping known processed SST: '{}'", - node.getFileName()); + currentNode.getFileName()); continue; } - if (destSnapFiles.contains(node.getFileName())) { + if (destSnapFiles.contains(currentNode.getFileName())) { LOG.debug("Src '{}' and dest '{}' have the same SST: '{}'", - src.getDbPath(), dest.getDbPath(), node.getFileName()); - sameFiles.add(node.getFileName()); + src.getDbPath(), dest.getDbPath(), currentNode.getFileName()); + sameFiles.add(currentNode.getFileName()); continue; } // Queue different SST to the next level LOG.debug("Src '{}' and dest '{}' have a different SST: '{}'", - src.getDbPath(), dest.getDbPath(), node.getFileName()); - nextLevel.add(node); + src.getDbPath(), dest.getDbPath(), currentNode.getFileName()); + nextLevel.add(currentNode); } } currentLevel = nextLevel; @@ -1159,8 +1218,8 @@ private CompactionNode addNodeToDAG(String file, String snapshotID, * arbitrary String as long as it helps debugging. * @param seqNum DB transaction sequence number. */ - private void populateCompactionDAG(List inputFiles, - List outputFiles, + private void populateCompactionDAG(List inputFiles, + List outputFiles, String snapshotId, long seqNum) { @@ -1168,13 +1227,13 @@ private void populateCompactionDAG(List inputFiles, LOG.debug("Input files: {} -> Output files: {}", inputFiles, outputFiles); } - for (FileInfo outfile : outputFiles) { + for (CompactionFileInfo outfile : outputFiles) { final CompactionNode outfileNode = compactionNodeMap.computeIfAbsent( outfile.getFileName(), file -> addNodeToDAG(file, snapshotId, seqNum, outfile.getStartKey(), outfile.getEndKey(), outfile.getColumnFamily())); - for (FileInfo infile : inputFiles) { + for (CompactionFileInfo infile : inputFiles) { final CompactionNode infileNode = compactionNodeMap.computeIfAbsent( infile.getFileName(), file -> addNodeToDAG(file, snapshotId, seqNum, infile.getStartKey(), infile.getEndKey(), @@ -1186,7 +1245,29 @@ private void populateCompactionDAG(List inputFiles, } } } + } + + private void addFileInfoToCompactionLogTable( + long dbSequenceNumber, + long creationTime, + String[] inputFiles, + String[] outputFiles, + String compactionReason + ) { + List inputFileInfoList = Arrays.stream(inputFiles) + .map(inputFile -> + new CompactionFileInfo(inputFile, null, null, null)) + .collect(Collectors.toList()); + List outputFileInfoList = Arrays.stream(outputFiles) + .map(outputFile -> + new CompactionFileInfo(outputFile, null, null, null)) + .collect(Collectors.toList()); + + CompactionLogEntry compactionLogEntry = + new CompactionLogEntry(dbSequenceNumber, creationTime, + inputFileInfoList, outputFileInfoList, compactionReason); + addToCompactionLogTable(compactionLogEntry); } /** @@ -1200,10 +1281,10 @@ public void pruneOlderSnapshotsWithCompactionHistory() { return; } - List olderSnapshotsLogFilePaths = - getOlderSnapshotsCompactionLogFilePaths(); - List lastCompactionSstFiles = - getLastCompactionSstFiles(olderSnapshotsLogFilePaths); + Pair, List> fileNodeToKeyPair = + getOlderSnapshotFileNodes(); + List lastCompactionSstFiles = fileNodeToKeyPair.getLeft(); + List keysToRemove = fileNodeToKeyPair.getRight(); Set sstFileNodesRemoved = pruneSstFileNodesFromDag(lastCompactionSstFiles); @@ -1215,85 +1296,66 @@ public void pruneOlderSnapshotsWithCompactionHistory() { try (BootstrapStateHandler.Lock lock = getBootstrapStateLock().lock()) { removeSstFiles(sstFileNodesRemoved); - deleteOlderSnapshotsCompactionFiles(olderSnapshotsLogFilePaths); + removeKeyFromCompactionLogFile(keysToRemove); } catch (InterruptedException e) { throw new RuntimeException(e); } } - /** - * Deletes the SST files from the backup directory if exists. - */ - private void removeSstFiles(Set sstFileNodes) { - for (String sstFileNode: sstFileNodes) { - File file = - new File(sstBackupDir + "/" + sstFileNode + SST_FILE_EXTENSION); - try { - Files.deleteIfExists(file.toPath()); - } catch (IOException exception) { - LOG.warn("Failed to delete SST file: " + sstFileNode, exception); - } - } - } - - /** - * Returns the list of compaction log files which are older than allowed - * max time in the compaction DAG. - */ - private List getOlderSnapshotsCompactionLogFilePaths() { + private Pair, List> getOlderSnapshotFileNodes() { long compactionLogPruneStartTime = System.currentTimeMillis(); + List compactionNodes = new ArrayList<>(); + List keysToRemove = new ArrayList<>(); - List compactionFiles = - listCompactionLogFileFromCompactionLogDirectory(); + try (ManagedRocksIterator managedRocksIterator = new ManagedRocksIterator( + activeRocksDB.newIterator(compactionLogTableCFHandle))) { + managedRocksIterator.get().seekToFirst(); + while (managedRocksIterator.get().isValid()) { + CompactionLogEntry compactionLogEntry = CompactionLogEntry + .getFromProtobuf(CompactionLogEntryProto + .parseFrom(managedRocksIterator.get().value())); - int index = compactionFiles.size() - 1; - for (; index >= 0; index--) { - Path compactionLogPath = compactionFiles.get(index); - SnapshotLogInfo snapshotLogInfo = - getSnapshotInfoFromLog(compactionLogPath); + if (compactionLogPruneStartTime - + compactionLogEntry.getCompactionTime() > maxAllowedTimeInDag) { + break; + } - if (snapshotLogInfo == null) { - continue; - } + compactionLogEntry.getInputFileInfoList() + .forEach(inputFileInfo -> + compactionNodes.add(inputFileInfo.getFileName())); + keysToRemove.add(managedRocksIterator.get().key()); + managedRocksIterator.get().next(); - if (compactionLogPruneStartTime - snapshotLogInfo.snapshotCreatedAt > - maxAllowedTimeInDag) { - break; } + } catch (InvalidProtocolBufferException exception) { + // TODO: Handle this properly before merging the PR. + throw new RuntimeException(exception); } + return Pair.of(compactionNodes, keysToRemove); + } - if (index >= 0) { - return compactionFiles.subList(0, index + 1); - } else { - return Collections.emptyList(); + public void removeKeyFromCompactionLogFile(List keysToRemove) { + try { + for (byte[] key: keysToRemove) { + activeRocksDB.delete(compactionLogTableCFHandle, key); + } + } catch (RocksDBException exception) { + // TODO Handle exception properly before merging the PR. + throw new RuntimeException(exception); } } /** - * Returns the list of compaction log file path from compaction log directory. + * Deletes the SST files from the backup directory if exists. */ - private List listCompactionLogFileFromCompactionLogDirectory() { - try (Stream pathStream = Files.list(Paths.get(compactionLogDir)) - .filter(e -> e.toString().toLowerCase() - .endsWith(COMPACTION_LOG_FILE_NAME_SUFFIX)) - .sorted()) { - return pathStream.collect(Collectors.toList()); - } catch (IOException e) { - throw new RuntimeException("Error listing compaction log dir " + - compactionLogDir, e); - } - } - - public void deleteOlderSnapshotsCompactionFiles( - List olderSnapshotsLogFilePaths) { - - for (int i = 0; i < olderSnapshotsLogFilePaths.size(); i++) { - Path olderSnapshotsLogFilePath = olderSnapshotsLogFilePaths.get(i); + private void removeSstFiles(Set sstFileNodes) { + for (String sstFileNode: sstFileNodes) { + File file = + new File(sstBackupDir + "/" + sstFileNode + SST_FILE_EXTENSION); try { - Files.deleteIfExists(olderSnapshotsLogFilePath); + Files.deleteIfExists(file.toPath()); } catch (IOException exception) { - LOG.error("Failed to deleted SST file: {}", olderSnapshotsLogFilePath, - exception); + LOG.warn("Failed to delete SST file: " + sstFileNode, exception); } } } @@ -1454,8 +1516,7 @@ private List getLastCompactionSstFiles(Path compactionLogFile) { try (Stream logStream = Files.lines(compactionLogFile, UTF_8)) { logStream.forEach(logLine -> { - if (!(logLine.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX) || - logLine.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX_V1))) { + if (!(logLine.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX))) { return; } sstFiles.set(logLine); @@ -1484,16 +1545,6 @@ private List getLastCompactionSstFiles(Path compactionLogFile) { String[] outputFiles = io[1].split(COMPACTION_LOG_ENTRY_FILE_DELIMITER); return asList(outputFiles); - } else if (lastCompactionLogEntry.startsWith( - COMPACTION_LOG_ENTRY_LINE_PREFIX_V1)) { - // Trim the beginning - lastCompactionLogEntry = lastCompactionLogEntry - .substring(COMPACTION_LOG_ENTRY_LINE_PREFIX_V1.length()); - CompactionLogEntry compactionLogEntry = - CompactionLogEntry.fromEncodedString(lastCompactionLogEntry); - return compactionLogEntry.getOutputFileInfoList().stream() - .map(FileInfo::getFileName) - .collect(Collectors.toList()); } else { throw new IllegalStateException("Should not have reached here. " + "Log line: " + lastCompactionLogEntry); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java index 2aee67e91cf1..9ff97751308e 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java @@ -225,6 +225,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager, public static final String SNAPSHOT_INFO_TABLE = "snapshotInfoTable"; public static final String SNAPSHOT_RENAMED_TABLE = "snapshotRenamedTable"; + public static final String COMPACTION_LOG_TABLE = + "compactionLogTable"; static final String[] ALL_TABLES = new String[] { USER_TABLE, @@ -247,7 +249,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager, PRINCIPAL_TO_ACCESS_IDS_TABLE, TENANT_STATE_TABLE, SNAPSHOT_INFO_TABLE, - SNAPSHOT_RENAMED_TABLE + SNAPSHOT_RENAMED_TABLE, + COMPACTION_LOG_TABLE }; private DBStore store; @@ -615,6 +618,7 @@ public static DBStoreBuilder addOMTablesAndCodecs(DBStoreBuilder builder) { .addTable(TENANT_STATE_TABLE) .addTable(SNAPSHOT_INFO_TABLE) .addTable(SNAPSHOT_RENAMED_TABLE) + .addTable(COMPACTION_LOG_TABLE) .addCodec(OzoneTokenIdentifier.class, TokenIdentifierCodec.get()) .addCodec(OmKeyInfo.class, OmKeyInfo.getCodec(true)) .addCodec(RepeatedOmKeyInfo.class, RepeatedOmKeyInfo.getCodec(true)) diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java index 1982e1b327e6..6e39be160eab 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java @@ -176,10 +176,10 @@ public BackgroundTaskResult call() throws Exception { LOG.debug("Processing snapshot {} to filter relevant SST Files", snapShotTableKey); - Map prefixPairs = getColumnFamilyToPrefixMap( - ozoneManager.getMetadataManager(), - snapshotInfo.getVolumeName(), - snapshotInfo.getBucketName()); + Map columnFamilyNameToPrefixMap = + getColumnFamilyToPrefixMap(ozoneManager.getMetadataManager(), + snapshotInfo.getVolumeName(), + snapshotInfo.getBucketName()); try ( ReferenceCounted @@ -191,7 +191,7 @@ public BackgroundTaskResult call() throws Exception { RocksDatabase db = rdbStore.getDb(); try (BootstrapStateHandler.Lock lock = getBootstrapStateLock() .lock()) { - db.deleteFilesNotMatchingPrefix(prefixPairs); + db.deleteFilesNotMatchingPrefix(columnFamilyNameToPrefixMap); } } catch (OMException ome) { // FILE_NOT_FOUND is obtained when the snapshot is deleted From a3663e0da6a22428a00370fa8007e092c4169c7e Mon Sep 17 00:00:00 2001 From: Hemant Kumar Date: Wed, 13 Sep 2023 15:41:12 -0700 Subject: [PATCH 3/6] updated unit tests --- .../ozone/rocksdiff/CompactionFileInfo.java | 19 +- .../rocksdiff/RocksDBCheckpointDiffer.java | 137 +-------- .../TestRocksDBCheckpointDiffer.java | 270 ++++++++---------- 3 files changed, 138 insertions(+), 288 deletions(-) diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionFileInfo.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionFileInfo.java index 8083ff12a115..90355797e7d8 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionFileInfo.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionFileInfo.java @@ -56,12 +56,19 @@ public String getColumnFamily() { } public HddsProtos.CompactionFileInfoProto getProtobuf() { - return HddsProtos.CompactionFileInfoProto.newBuilder() - .setFileName(fileName) - .setStartKey(startKey) - .setEndKey(endKey) - .setColumnFamily(columnFamily) - .build(); + HddsProtos.CompactionFileInfoProto.Builder builder = + HddsProtos.CompactionFileInfoProto.newBuilder() + .setFileName(fileName); + if (startKey != null) { + builder = builder.setStartKey(startKey); + } + if (endKey != null) { + builder = builder.setStartKey(endKey); + } + if (columnFamily != null) { + builder = builder.setStartKey(columnFamily); + } + return builder.build(); } public static CompactionFileInfo getFromProtobuf( diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java index 3f0e2bb5281d..231753f5b57b 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java @@ -24,13 +24,11 @@ import java.io.FileNotFoundException; import java.util.Arrays; -import java.util.Collections; import java.util.Objects; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import com.google.protobuf.InvalidProtocolBufferException; import org.apache.commons.collections.CollectionUtils; @@ -82,7 +80,6 @@ import org.apache.hadoop.ozone.lock.BootstrapStateHandler; import static java.nio.charset.StandardCharsets.UTF_8; -import static java.util.Arrays.asList; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED_DEFAULT; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL; @@ -177,10 +174,8 @@ public class RocksDBCheckpointDiffer implements AutoCloseable, /** * Used during DAG reconstruction. */ - private long reconstructionSnapshotGeneration; private long reconstructionSnapshotCreationTime; private String reconstructionCompactionReason; - private String reconstructionLastSnapshotID; private final Scheduler scheduler; private volatile boolean closed; @@ -341,7 +336,6 @@ public void setCurrentCompactionLog(long latestSequenceNum) { appendToCurrentCompactionLog(""); } - @Override public void close() throws Exception { if (!closed) { @@ -778,10 +772,8 @@ void processCompactionLogLine(String line) { reconstructionCompactionReason = line.substring(COMPACTION_LOG_COMMENT_LINE_PREFIX.length()); } else if (line.startsWith(COMPACTION_LOG_SEQ_NUM_LINE_PREFIX)) { - SnapshotLogInfo snapshotLogInfo = getSnapshotLogInfo(line); - reconstructionSnapshotGeneration = snapshotLogInfo.snapshotGenerationId; - reconstructionLastSnapshotID = snapshotLogInfo.snapshotId; - reconstructionSnapshotCreationTime = snapshotLogInfo.snapshotCreatedAt; + reconstructionSnapshotCreationTime = + getSnapshotCreationTimeFromLogLing(line); } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) { // Compaction log entry is like following: // C sequence_number input_files:output_files @@ -792,6 +784,7 @@ void processCompactionLogLine(String line) { return; } + String dbSequenceNumber = lineSpilt[1]; String[] io = lineSpilt[2] .split(COMPACTION_LOG_ENTRY_INPUT_OUTPUT_FILES_DELIMITER); @@ -806,7 +799,7 @@ void processCompactionLogLine(String line) { String[] inputFiles = io[0].split(COMPACTION_LOG_ENTRY_FILE_DELIMITER); String[] outputFiles = io[1].split(COMPACTION_LOG_ENTRY_FILE_DELIMITER); - addFileInfoToCompactionLogTable(reconstructionSnapshotGeneration, + addFileInfoToCompactionLogTable(Long.parseLong(dbSequenceNumber), reconstructionSnapshotCreationTime, inputFiles, outputFiles, reconstructionCompactionReason); } else { @@ -839,7 +832,6 @@ public void addEntriesFromLogFilesToDagAndCompactionLogTable() { throw new RuntimeException("Compaction log directory must be set " + "first"); } - reconstructionSnapshotGeneration = 0L; reconstructionSnapshotCreationTime = 0L; reconstructionCompactionReason = null; try { @@ -851,7 +843,7 @@ public void addEntriesFromLogFilesToDagAndCompactionLogTable() { readCompactionLogFile(logPath.toString()); // Delete the file once entries are added to compaction table // so that on next restart, only compaction log table is used. - Files.deleteIfExists(logPath); + Files.delete(logPath); } } } catch (IOException e) { @@ -1296,7 +1288,7 @@ public void pruneOlderSnapshotsWithCompactionHistory() { try (BootstrapStateHandler.Lock lock = getBootstrapStateLock().lock()) { removeSstFiles(sstFileNodesRemoved); - removeKeyFromCompactionLogFile(keysToRemove); + removeKeyFromCompactionLogTable(keysToRemove); } catch (InterruptedException e) { throw new RuntimeException(e); } @@ -1316,7 +1308,7 @@ private Pair, List> getOlderSnapshotFileNodes() { .parseFrom(managedRocksIterator.get().value())); if (compactionLogPruneStartTime - - compactionLogEntry.getCompactionTime() > maxAllowedTimeInDag) { + compactionLogEntry.getCompactionTime() < maxAllowedTimeInDag) { break; } @@ -1334,7 +1326,8 @@ private Pair, List> getOlderSnapshotFileNodes() { return Pair.of(compactionNodes, keysToRemove); } - public void removeKeyFromCompactionLogFile(List keysToRemove) { + private synchronized void removeKeyFromCompactionLogTable( + List keysToRemove) { try { for (byte[] key: keysToRemove) { activeRocksDB.delete(compactionLogTableCFHandle, key); @@ -1445,29 +1438,7 @@ Set pruneForwardDag(MutableGraph forwardDag, return removedFiles; } - private SnapshotLogInfo getSnapshotInfoFromLog(Path compactionLogFile) { - AtomicReference snapshotLogInfo = - new AtomicReference<>(); - try (Stream logStream = Files.lines(compactionLogFile, UTF_8)) { - logStream.forEach(logLine -> { - if (!logLine.startsWith(COMPACTION_LOG_SEQ_NUM_LINE_PREFIX)) { - return; - } - - snapshotLogInfo.set(getSnapshotLogInfo(logLine)); - }); - } catch (IOException exception) { - throw new RuntimeException("Failed to read compaction log file: " + - compactionLogFile, exception); - } - - return snapshotLogInfo.get(); - } - - /** - * Converts a snapshot compaction log line to SnapshotLogInfo. - */ - private SnapshotLogInfo getSnapshotLogInfo(String logLine) { + private long getSnapshotCreationTimeFromLogLing(String logLine) { // Remove `S ` from the line. String line = logLine.substring(COMPACTION_LOG_SEQ_NUM_LINE_PREFIX.length()); @@ -1476,79 +1447,7 @@ private SnapshotLogInfo getSnapshotLogInfo(String logLine) { Preconditions.checkArgument(splits.length == 3, "Snapshot info log statement has more than expected parameters."); - return new SnapshotLogInfo(Long.parseLong(splits[0]), - splits[1], - Long.parseLong(splits[2])); - } - - /** - * Returns the list of SST files got compacted in the last compaction from - * the provided list of compaction log files. - * We can't simply use last file from the list because it is possible that - * no compaction happened between the last snapshot and previous to that. - * Hence, we go over the list in reverse order and return the SST files from - * first the compaction happened in the reverse list. - * If no compaction happen at all, it returns empty list. - */ - private List getLastCompactionSstFiles( - List compactionLogFiles - ) { - - if (compactionLogFiles.isEmpty()) { - return Collections.emptyList(); - } - compactionLogFiles = new ArrayList<>(compactionLogFiles); - Collections.reverse(compactionLogFiles); - - for (Path compactionLogFile: compactionLogFiles) { - List sstFiles = getLastCompactionSstFiles(compactionLogFile); - if (!sstFiles.isEmpty()) { - return sstFiles; - } - } - - return Collections.emptyList(); - } - - private List getLastCompactionSstFiles(Path compactionLogFile) { - - AtomicReference sstFiles = new AtomicReference<>(); - - try (Stream logStream = Files.lines(compactionLogFile, UTF_8)) { - logStream.forEach(logLine -> { - if (!(logLine.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX))) { - return; - } - sstFiles.set(logLine); - }); - } catch (IOException exception) { - throw new RuntimeException("Failed to read file: " + compactionLogFile, - exception); - } - - String lastCompactionLogEntry = sstFiles.get(); - - if (StringUtils.isEmpty(lastCompactionLogEntry)) { - return Collections.emptyList(); - } - - if (lastCompactionLogEntry.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) { - // Trim the beginning - lastCompactionLogEntry = lastCompactionLogEntry - .substring(COMPACTION_LOG_ENTRY_LINE_PREFIX.length()); - - String[] io = lastCompactionLogEntry - .split(COMPACTION_LOG_ENTRY_INPUT_OUTPUT_FILES_DELIMITER); - - assert (io.length == 2); - - String[] outputFiles = io[1].split(COMPACTION_LOG_ENTRY_FILE_DELIMITER); - - return asList(outputFiles); - } else { - throw new IllegalStateException("Should not have reached here. " + - "Log line: " + lastCompactionLogEntry); - } + return Long.parseLong(splits[2]); } public String getSSTBackupDir() { @@ -1559,20 +1458,6 @@ public String getCompactionLogDir() { return compactionLogDir; } - private static final class SnapshotLogInfo { - private final long snapshotGenerationId; - private final String snapshotId; - private final long snapshotCreatedAt; - - private SnapshotLogInfo(long snapshotGenerationId, - String snapshotId, - long snapshotCreatedAt) { - this.snapshotGenerationId = snapshotGenerationId; - this.snapshotId = snapshotId; - this.snapshotCreatedAt = snapshotCreatedAt; - } - } - /** * Defines the task that removes SST files from backup directory which are * not needed to generate snapshot diff using compaction DAG to clean diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java index b550445c35f0..48fce50b9db8 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java @@ -51,7 +51,9 @@ import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB; +import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; import org.apache.hadoop.ozone.lock.BootstrapStateHandler; import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.NodeComparator; import org.apache.ozone.test.GenericTestUtils; @@ -121,9 +123,13 @@ public class TestRocksDBCheckpointDiffer { private File sstBackUpDir; private ConfigurationSource config; private ExecutorService executorService = Executors.newCachedThreadPool(); + private RocksDBCheckpointDiffer rocksDBCheckpointDiffer; + private RocksDB activeRocksDB; + private ColumnFamilyHandle keyTableCFHandle; + private ColumnFamilyHandle compactionLogTableCFHandle; @BeforeEach - public void init() { + public void init() throws RocksDBException { // Checkpoint differ log level. Set to DEBUG for verbose output GenericTestUtils.setLogLevel(RocksDBCheckpointDiffer.getLog(), Level.INFO); // Test class log level. Set to DEBUG for verbose output @@ -152,6 +158,32 @@ public void init() { OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL, OZONE_OM_SNAPSHOT_PRUNE_COMPACTION_DAG_DAEMON_RUN_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS)).thenReturn(0L); + + rocksDBCheckpointDiffer = new RocksDBCheckpointDiffer(metadataDirName, + sstBackUpDirName, + compactionLogDirName, + activeDbDirName, + config); + + ColumnFamilyOptions cfOpts = new ColumnFamilyOptions() + .optimizeUniversalStyleCompaction(); + List cfDescriptors = getCFDescriptorList(cfOpts); + List cfHandles = new ArrayList<>(); + DBOptions dbOptions = new DBOptions() + .setCreateIfMissing(true) + .setCreateMissingColumnFamilies(true); + + rocksDBCheckpointDiffer.setRocksDBForCompactionTracking(dbOptions); + activeRocksDB = RocksDB.open(dbOptions, activeDbDirName, cfDescriptors, + cfHandles); + keyTableCFHandle = cfHandles.get(1); + compactionLogTableCFHandle = cfHandles.get(4); + + rocksDBCheckpointDiffer.setCurrentCompactionLog( + activeRocksDB.getLatestSequenceNumber()); + rocksDBCheckpointDiffer.setCompactionLogTableCFHandle(cfHandles.get(4)); + rocksDBCheckpointDiffer.setActiveRocksDB(activeRocksDB); + rocksDBCheckpointDiffer.loadAllCompactionLogs(); } private void createDir(File file, String filePath) { @@ -168,6 +200,10 @@ private void createDir(File file, String filePath) { @AfterEach public void cleanUp() { + IOUtils.closeQuietly(rocksDBCheckpointDiffer); + IOUtils.closeQuietly(keyTableCFHandle); + IOUtils.closeQuietly(compactionLogTableCFHandle); + IOUtils.closeQuietly(activeRocksDB); deleteDirectory(compactionLogDir); deleteDirectory(sstBackUpDir); deleteDirectory(metadataDirDir); @@ -274,12 +310,6 @@ public void testGetSSTDiffListWithoutDB(String description, Set expectedDiffSstFiles, boolean expectingException) { - RocksDBCheckpointDiffer differ = - new RocksDBCheckpointDiffer(metadataDirName, - sstBackUpDirName, - compactionLogDirName, - activeDbDirName, - config); boolean exceptionThrown = false; long createdTime = System.currentTimeMillis(); @@ -309,13 +339,14 @@ public void testGetSSTDiffListWithoutDB(String description, // Construct DAG from compaction log input Arrays.stream(compactionLog.split("\n")).forEach( - differ::processCompactionLogLine); + rocksDBCheckpointDiffer::processCompactionLogLine); + rocksDBCheckpointDiffer.loadAllCompactionLogs(); Set actualSameSstFiles = new HashSet<>(); Set actualDiffSstFiles = new HashSet<>(); try { - differ.internalGetSSTDiffList( + rocksDBCheckpointDiffer.internalGetSSTDiffList( srcSnapshot, destSnapshot, srcSnapshotSstFiles, @@ -348,44 +379,37 @@ public void testGetSSTDiffListWithoutDB(String description, */ @Test void testDifferWithDB() throws Exception { - RocksDBCheckpointDiffer differ = - new RocksDBCheckpointDiffer(metadataDirName, - sstBackUpDirName, - compactionLogDirName, - activeDbDirName, - config); - RocksDB rocksDB = - createRocksDBInstanceAndWriteKeys(activeDbDirName, differ); - readRocksDBInstance(activeDbDirName, rocksDB, null, differ); + writeKeysAndCheckpointing(); + readRocksDBInstance(activeDbDirName, activeRocksDB, null, + rocksDBCheckpointDiffer); if (LOG.isDebugEnabled()) { printAllSnapshots(); } - traverseGraph(differ.getCompactionNodeMap(), - differ.getBackwardCompactionDAG(), - differ.getForwardCompactionDAG()); + traverseGraph(rocksDBCheckpointDiffer.getCompactionNodeMap(), + rocksDBCheckpointDiffer.getBackwardCompactionDAG(), + rocksDBCheckpointDiffer.getForwardCompactionDAG()); - diffAllSnapshots(differ); + diffAllSnapshots(rocksDBCheckpointDiffer); // Confirm correct links created try (Stream sstPathStream = Files.list(sstBackUpDir.toPath())) { List expectedLinks = sstPathStream.map(Path::getFileName) .map(Object::toString).sorted().collect(Collectors.toList()); Assertions.assertEquals(expectedLinks, asList( - "000015.sst", "000017.sst", "000019.sst", "000021.sst", - "000022.sst", "000024.sst", "000026.sst")); + "000017.sst", "000019.sst", "000021.sst", "000023.sst", + "000024.sst", "000026.sst", "000029.sst")); } if (LOG.isDebugEnabled()) { - differ.dumpCompactionNodeTable(); + rocksDBCheckpointDiffer.dumpCompactionNodeTable(); } - rocksDB.close(); - cleanUp(); + cleanUpSnapshots(); } - public void cleanup() { + public void cleanUpSnapshots() { for (DifferSnapshotInfo snap : snapshots) { snap.getRocksDB().close(); } @@ -412,12 +436,12 @@ void diffAllSnapshots(RocksDBCheckpointDiffer differ) // Hard-coded expected output. // The results are deterministic. Retrieved from a successful run. final List> expectedDifferResult = asList( - asList("000024", "000017", "000028", "000026", "000019", "000021"), - asList("000024", "000028", "000026", "000019", "000021"), - asList("000024", "000028", "000026", "000021"), - asList("000024", "000028", "000026"), - asList("000028", "000026"), - Collections.singletonList("000028"), + asList("000023", "000029", "000026", "000019", "000021", "000031"), + asList("000023", "000029", "000026", "000021", "000031"), + asList("000023", "000029", "000026", "000031"), + asList("000029", "000026", "000031"), + asList("000029", "000031"), + Collections.singletonList("000031"), Collections.emptyList() ); Assertions.assertEquals(snapshots.size(), expectedDifferResult.size()); @@ -465,11 +489,6 @@ private void createCheckpoint(RocksDBCheckpointDiffer differ, colHandle)); this.snapshots.add(currentSnapshot); - // Same as what OmSnapshotManager#createOmSnapshotCheckpoint would do - differ.appendSnapshotInfoToCompactionLog(dbLatestSequenceNumber, - snapshotId.toString(), - System.currentTimeMillis()); - differ.setCurrentCompactionLog(dbLatestSequenceNumber); long t2 = System.currentTimeMillis(); @@ -507,52 +526,24 @@ static List getCFDescriptorList( new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, cfOpts), new ColumnFamilyDescriptor("keyTable".getBytes(UTF_8), cfOpts), new ColumnFamilyDescriptor("directoryTable".getBytes(UTF_8), cfOpts), - new ColumnFamilyDescriptor("fileTable".getBytes(UTF_8), cfOpts) + new ColumnFamilyDescriptor("fileTable".getBytes(UTF_8), cfOpts), + new ColumnFamilyDescriptor("compactionLogTable".getBytes(UTF_8), cfOpts) ); } - // Test Code to create sample RocksDB instance. - private RocksDB createRocksDBInstanceAndWriteKeys(String dbPathArg, - RocksDBCheckpointDiffer differ) throws RocksDBException { - - LOG.debug("Creating RocksDB at '{}'", dbPathArg); - - // Delete the test DB dir if it exists - File dir = new File(dbPathArg); - if (dir.exists()) { - deleteDirectory(dir); - } - - final ColumnFamilyOptions cfOpts = new ColumnFamilyOptions() - .optimizeUniversalStyleCompaction(); - final List cfDescriptors = - getCFDescriptorList(cfOpts); - List cfHandles = new ArrayList<>(); - - // Create a RocksDB instance with compaction tracking - final DBOptions dbOptions = new DBOptions() - .setCreateIfMissing(true) - .setCreateMissingColumnFamilies(true); - differ.setRocksDBForCompactionTracking(dbOptions); - RocksDB rocksDB = RocksDB.open(dbOptions, dbPathArg, cfDescriptors, - cfHandles); - - differ.setCurrentCompactionLog(rocksDB.getLatestSequenceNumber()); - - // key-value + private void writeKeysAndCheckpointing() throws RocksDBException { for (int i = 0; i < NUM_ROW; ++i) { String generatedString = RandomStringUtils.randomAlphabetic(7); String keyStr = "Key-" + i + "-" + generatedString; String valueStr = "Val-" + i + "-" + generatedString; byte[] key = keyStr.getBytes(UTF_8); // Put entry in keyTable - rocksDB.put(cfHandles.get(1), key, valueStr.getBytes(UTF_8)); + activeRocksDB.put(keyTableCFHandle, key, valueStr.getBytes(UTF_8)); if (i % SNAPSHOT_EVERY_SO_MANY_KEYS == 0) { - createCheckpoint(differ, rocksDB); + createCheckpoint(rocksDBCheckpointDiffer, activeRocksDB); } } - createCheckpoint(differ, rocksDB); - return rocksDB; + createCheckpoint(rocksDBCheckpointDiffer, activeRocksDB); } private boolean deleteDirectory(File directoryToBeDeleted) { @@ -887,15 +878,8 @@ public void testPruneBackwardDag(String description, Set levelToBeRemoved, MutableGraph expectedDag, Set expectedFileNodesRemoved) { - - RocksDBCheckpointDiffer differ = - new RocksDBCheckpointDiffer(metadataDirName, - sstBackUpDirName, - compactionLogDirName, - activeDbDirName, - config); Set actualFileNodesRemoved = - differ.pruneBackwardDag(originalDag, levelToBeRemoved); + rocksDBCheckpointDiffer.pruneBackwardDag(originalDag, levelToBeRemoved); Assertions.assertEquals(expectedDag, originalDag); Assertions.assertEquals(actualFileNodesRemoved, expectedFileNodesRemoved); } @@ -949,15 +933,8 @@ public void testPruneForwardDag(String description, Set levelToBeRemoved, MutableGraph expectedDag, Set expectedFileNodesRemoved) { - - RocksDBCheckpointDiffer differ = - new RocksDBCheckpointDiffer(metadataDirName, - sstBackUpDirName, - compactionLogDirName, - activeDbDirName, - config); Set actualFileNodesRemoved = - differ.pruneForwardDag(originalDag, levelToBeRemoved); + rocksDBCheckpointDiffer.pruneForwardDag(originalDag, levelToBeRemoved); Assertions.assertEquals(expectedDag, originalDag); Assertions.assertEquals(actualFileNodesRemoved, expectedFileNodesRemoved); } @@ -967,38 +944,38 @@ private static Stream compactionDagPruningScenarios() { String compactionLogFile0 = "S 1000 snapshotId0 " + (currentTimeMillis - MINUTES.toMillis(30)) + " \n"; - String compactionLogFile1 = "C 1000 000015,000013,000011,000009:000018," + + String compactionLogFile1 = "C 1500 000015,000013,000011,000009:000018," + "000016,000017\n" + "S 2000 snapshotId1 " + (currentTimeMillis - MINUTES.toMillis(24)) + " \n"; - String compactionLogFile2 = "C 1000 000018,000016,000017,000026,000024," + + String compactionLogFile2 = "C 2500 000018,000016,000017,000026,000024," + "000022,000020:000027,000030,000028,000031,000029\n" + "S 3000 snapshotId2 " + (currentTimeMillis - MINUTES.toMillis(18)) + " \n"; - String compactionLogFile3 = "C 1000 000027,000030,000028,000031,000029," + + String compactionLogFile3 = "C 3500 000027,000030,000028,000031,000029," + "000039,000037,000035,000033:000040,000044,000042,000043,000046," + "000041,000045\n" - + "S 3000 snapshotId3 " + + + "S 4000 snapshotId3 " + (currentTimeMillis - MINUTES.toMillis(12)) + " \n"; - String compactionLogFile4 = "C 1000 000040,000044,000042,000043,000046," + + String compactionLogFile4 = "C 4500 000040,000044,000042,000043,000046," + "000041,000045,000054,000052,000050,000048:000059,000055,000056," + "000060,000057,000058\n" - + "S 3000 snapshotId4 " + + + "S 5000 snapshotId4 " + (currentTimeMillis - MINUTES.toMillis(6)) + " \n"; - String compactionLogFileWithoutSnapshot1 = "C 1000 000015,000013,000011," + + String compactionLogFileWithoutSnapshot1 = "C 1500 000015,000013,000011," + "000009:000018,000016,000017\n" + "C 2000 000018,000016,000017,000026,000024,000022,000020" + ":000027,000030,000028,000031,000029\n"; - String compactionLogFileWithoutSnapshot2 = "C 3000 000027,000030,000028," + + String compactionLogFileWithoutSnapshot2 = "C 3500 000027,000030,000028," + "000031,000029,000039,000037,000035,000033:000040,000044,000042," + "000043,000046,000041,000045\n"; - String compactionLogFileWithoutSnapshot3 = "C 4000 000040,000044,000042," + + String compactionLogFileWithoutSnapshot3 = "C 4500 000040,000044,000042," + "000043,000046,000041,000045,000054,000052,000050,000048:000059," + "000055,000056,000060,000057,000058\n"; @@ -1027,18 +1004,8 @@ private static Stream compactionDagPruningScenarios() { (currentTimeMillis - MINUTES.toMillis(3)) + " \n"; Set expectedNodes = new HashSet<>( - Arrays.asList("000054", "000052", "000050", "000048", "000059", - "000055", "000056", "000060", "000057", "000058") - ); - - Set expectedAllNodes = new HashSet<>( - Arrays.asList("000058", "000013", "000035", "000057", "000056", - "000011", "000033", "000055", "000018", "000017", "000039", - "000016", "000015", "000037", "000059", "000060", "000043", - "000020", "000042", "000041", "000040", "000024", "000046", - "000045", "000022", "000044", "000029", "000028", "000027", - "000026", "000048", "000009", "000050", "000054", "000031", - "000030", "000052") + Arrays.asList("000059", "000055", "000056", "000060", "000057", + "000058") ); return Stream.of( @@ -1047,7 +1014,8 @@ private static Stream compactionDagPruningScenarios() { Arrays.asList(compactionLogFile0, compactionLogFile1, compactionLogFile2, compactionLogFile3, compactionLogFile4), expectedNodes, - 4 + 4, + 0 ), Arguments.of("Compaction log doesn't have snapshot because OM" + " restarted. Restart happened before snapshot to be deleted.", @@ -1056,7 +1024,8 @@ private static Stream compactionDagPruningScenarios() { compactionLogFile3, compactionLogFile4), expectedNodes, - 3 + 4, + 0 ), Arguments.of("Compaction log doesn't have snapshot because OM" + " restarted. Restart happened after snapshot to be deleted.", @@ -1065,7 +1034,8 @@ private static Stream compactionDagPruningScenarios() { compactionLogFileWithoutSnapshot3, compactionLogFileOnlyWithSnapshot4), expectedNodes, - 4 + 4, + 0 ), Arguments.of("No compaction happened in between two snapshots.", Arrays.asList(compactionLogFile0, compactionLogFile1, @@ -1073,21 +1043,7 @@ private static Stream compactionDagPruningScenarios() { compactionLogFileOnlyWithSnapshot1, compactionLogFileOnlyWithSnapshot2, compactionLogFile4), expectedNodes, - 6 - ), - Arguments.of("No snapshot is taken and only one compaction log file,", - Collections.singletonList(compactionLogFileWithoutSnapshot1 + - compactionLogFileWithoutSnapshot2 + - compactionLogFileWithoutSnapshot3), - expectedAllNodes, - 0 - ), - Arguments.of("No snapshot is taken but multiple compaction files" + - " because of OM restart.", - Arrays.asList(compactionLogFileWithoutSnapshot1, - compactionLogFileWithoutSnapshot2, - compactionLogFileWithoutSnapshot3), - expectedAllNodes, + 4, 0 ), Arguments.of("Only contains snapshots but no compaction.", @@ -1098,12 +1054,14 @@ private static Stream compactionDagPruningScenarios() { compactionLogFileOnlyWithSnapshot5, compactionLogFileOnlyWithSnapshot6), Collections.emptySet(), - 3 + 0, + 0 ), Arguments.of("No file exists because compaction has not happened" + " and snapshot is not taken.", Collections.emptyList(), Collections.emptySet(), + 0, 0 ) ); @@ -1118,7 +1076,8 @@ public void testPruneOlderSnapshotsWithCompactionHistory( String description, List compactionLogs, Set expectedNodes, - int expectedNumberOfLogFilesDeleted + int expectedNumberOfLogEntriesBeforePruning, + int expectedNumberOfLogEntriesAfterPruning ) throws IOException, ExecutionException, InterruptedException, TimeoutException { List filesCreated = new ArrayList<>(); @@ -1132,25 +1091,21 @@ public void testPruneOlderSnapshotsWithCompactionHistory( filesCreated.add(compactionFile); } - RocksDBCheckpointDiffer differ = - new RocksDBCheckpointDiffer(metadataDirName, - sstBackUpDirName, - compactionLogDirName, - activeDbDirName, - config); - - differ.loadAllCompactionLogs(); - - waitForLock(differ, + rocksDBCheckpointDiffer.loadAllCompactionLogs(); + assertEquals(expectedNumberOfLogEntriesBeforePruning, + countEntriesInCompactionLogTable()); + waitForLock(rocksDBCheckpointDiffer, RocksDBCheckpointDiffer::pruneOlderSnapshotsWithCompactionHistory); - Set actualNodesInForwardDAG = differ.getForwardCompactionDAG() + Set actualNodesInForwardDAG = rocksDBCheckpointDiffer + .getForwardCompactionDAG() .nodes() .stream() .map(CompactionNode::getFileName) .collect(Collectors.toSet()); - Set actualNodesBackwardDAG = differ.getBackwardCompactionDAG() + Set actualNodesBackwardDAG = rocksDBCheckpointDiffer + .getBackwardCompactionDAG() .nodes() .stream() .map(CompactionNode::getFileName) @@ -1159,15 +1114,24 @@ public void testPruneOlderSnapshotsWithCompactionHistory( assertEquals(expectedNodes, actualNodesInForwardDAG); assertEquals(expectedNodes, actualNodesBackwardDAG); - for (int i = 0; i < expectedNumberOfLogFilesDeleted; i++) { + for (int i = 0; i < compactionLogs.size(); i++) { File compactionFile = filesCreated.get(i); assertFalse(compactionFile.exists()); } + assertEquals(expectedNumberOfLogEntriesAfterPruning, + countEntriesInCompactionLogTable()); + } - for (int i = expectedNumberOfLogFilesDeleted; i < compactionLogs.size(); - i++) { - File compactionFile = filesCreated.get(i); - assertTrue(compactionFile.exists()); + private int countEntriesInCompactionLogTable() { + try (ManagedRocksIterator iterator = new ManagedRocksIterator( + activeRocksDB.newIterator(compactionLogTableCFHandle))) { + iterator.get().seekToFirst(); + int count = 0; + while (iterator.get().isValid()) { + iterator.get().next(); + count++; + } + return count; } } @@ -1252,16 +1216,10 @@ public void testSstFilePruning( fileName); } - RocksDBCheckpointDiffer differ = - new RocksDBCheckpointDiffer(metadataDirName, - sstBackUpDirName, - compactionLogDirName, - activeDbDirName, - config); - - differ.loadAllCompactionLogs(); + rocksDBCheckpointDiffer.loadAllCompactionLogs(); - waitForLock(differ, RocksDBCheckpointDiffer::pruneSstFiles); + waitForLock(rocksDBCheckpointDiffer, + RocksDBCheckpointDiffer::pruneSstFiles); Set actualFileSetAfterPruning; try (Stream pathStream = Files.list( From 24ee852ce8d8c6c02fb470e18d6b3a25dbbb1e67 Mon Sep 17 00:00:00 2001 From: Hemant Kumar Date: Thu, 14 Sep 2023 10:40:06 -0700 Subject: [PATCH 4/6] minor fixes --- .../ozone/rocksdiff/CompactionLogEntry.java | 13 ++++++------- .../apache/ozone/rocksdiff/CompactionNode.java | 10 ++++++---- .../rocksdiff/RocksDBCheckpointDiffer.java | 17 ++++++----------- 3 files changed, 18 insertions(+), 22 deletions(-) diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java index 106f928d835f..af7e7ce20a44 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java @@ -114,12 +114,12 @@ public static CompactionLogEntry getFromProtobuf( @Override public String toString() { - return String.format("dbSequenceNumber: '%s', inputFileInfoList: '%s'," + - " outputFileInfoList: '%s',", dbSequenceNumber, inputFileInfoList, - outputFileInfoList); + return String.format("dbSequenceNumber: '%s', compactionTime: '%s', " + + "inputFileInfoList: '%s', outputFileInfoList: '%s', " + + "compactionReason: '%s'.", dbSequenceNumber, compactionTime, + inputFileInfoList, outputFileInfoList, compactionReason); } - /** * Builder of CompactionLogEntry. */ @@ -193,9 +193,8 @@ private List toFileInfoList( iterator.seekToLast(); String endKey = StringUtils.bytes2String(iterator.key()); - CompactionFileInfo - fileInfo = new CompactionFileInfo(fileName, startKey, endKey, - columnFamily); + CompactionFileInfo fileInfo = + new CompactionFileInfo(fileName, startKey, endKey, columnFamily); response.add(fileInfo); } catch (RocksDBException rocksDBException) { throw new RuntimeException("Failed to read SST file: " + sstFile, diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionNode.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionNode.java index a1faf2158173..1de075c4965c 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionNode.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionNode.java @@ -38,17 +38,19 @@ public class CompactionNode { * @param ssId snapshotId field. Added here for improved debuggability only * @param numKeys Number of keys in the SST * @param seqNum Snapshot generation (sequence number) + * @param startKey start key in the SST file. + * @param endKey end key in the SST file. + * @param columnFamily column family SST file belongs to. */ public CompactionNode(String file, String ssId, long numKeys, long seqNum, - String keyStartRange, String keyEndRange, - String columnFamily) { + String startKey, String endKey, String columnFamily) { this.fileName = file; this.snapshotId = ssId; this.totalNumberOfKeys = numKeys; this.snapshotGeneration = seqNum; this.cumulativeKeysReverseTraversal = 0L; - this.startKey = keyStartRange; - this.endKey = keyEndRange; + this.startKey = startKey; + this.endKey = endKey; this.columnFamily = columnFamily; } diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java index 231753f5b57b..46fb8c179576 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java @@ -613,11 +613,6 @@ public void onCompactionCompleted(RocksDB db, } private void addToCompactionLogTable(CompactionLogEntry compactionLogEntry) { - // This is temporary till tests are fixed. - if (compactionLogTableCFHandle == null) { - return; - } - // Key in the transactionId-currentTime // Just trxId can't be used because multiple compaction might be // running, and it is possible no new entry was added to DB. @@ -1184,8 +1179,8 @@ MutableGraph getBackwardCompactionDAG() { * @return CompactionNode */ private CompactionNode addNodeToDAG(String file, String snapshotID, - long seqNum, String startRange, - String endRange, String columnFamily) { + long seqNum, String startKey, + String endKey, String columnFamily) { long numKeys = 0L; try { numKeys = getSSTFileSummary(file); @@ -1195,7 +1190,7 @@ private CompactionNode addNodeToDAG(String file, String snapshotID, LOG.info("Can't find SST '{}'", file); } CompactionNode fileNode = new CompactionNode(file, snapshotID, numKeys, - seqNum, startRange, endRange, columnFamily); + seqNum, startKey, endKey, columnFamily); forwardCompactionDAG.addNode(fileNode); backwardCompactionDAG.addNode(fileNode); @@ -1595,9 +1590,9 @@ public void pngPrintMutableGraph(String filePath, GraphType graphType) private boolean shouldSkipNode(CompactionNode node, Map columnFamilyToPrefixMap) { - // This is for backward compatibility. Before the compaction log V1 - // migration,startKey, endKey and columnFamily information is not persisted - // in compaction logs. + // This is for backward compatibility. Before the compaction log table + // migration, startKey, endKey and columnFamily information is not persisted + // in compaction log files. if (node.getStartKey() == null || node.getEndKey() == null || node.getColumnFamily() == null) { LOG.debug("Compaction node with fileName: {} doesn't have startKey, " + From 90b32c2ea945a6a6f5134dc019c7be5a2d733a69 Mon Sep 17 00:00:00 2001 From: Hemant Kumar Date: Thu, 14 Sep 2023 12:13:37 -0700 Subject: [PATCH 5/6] Fixed unit tests --- .../ozone/rocksdiff/CompactionLogEntry.java | 45 ++++++++++++++++++- .../rocksdiff/RocksDBCheckpointDiffer.java | 12 +++-- .../ozone/om/OmMetadataManagerImpl.java | 10 ++++- .../hadoop/ozone/om/codec/OMDBDefinition.java | 11 +++++ 4 files changed, 72 insertions(+), 6 deletions(-) diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java index af7e7ce20a44..bc885bb8756b 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java @@ -21,6 +21,10 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.CompactionLogEntryProto; +import org.apache.hadoop.hdds.utils.db.Codec; +import org.apache.hadoop.hdds.utils.db.CopyObject; +import org.apache.hadoop.hdds.utils.db.DelegatedCodec; +import org.apache.hadoop.hdds.utils.db.Proto2Codec; import org.apache.hadoop.hdds.utils.db.managed.ManagedOptions; import org.apache.hadoop.hdds.utils.db.managed.ManagedReadOptions; import org.rocksdb.RocksDBException; @@ -30,6 +34,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Objects; import java.util.stream.Collectors; import static org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.SST_FILE_EXTENSION_LENGTH; @@ -37,7 +42,16 @@ /** * Compaction log entry Dao to write to the compaction log file. */ -public class CompactionLogEntry { +public class CompactionLogEntry implements CopyObject { + private static final Codec CODEC = new DelegatedCodec<>( + Proto2Codec.get(CompactionLogEntryProto.class), + CompactionLogEntry::getFromProtobuf, + CompactionLogEntry::getProtobuf); + + public static Codec getCodec() { + return CODEC; + } + private final long dbSequenceNumber; private final long compactionTime; private final List inputFileInfoList; @@ -204,4 +218,33 @@ private List toFileInfoList( return response; } } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof CompactionLogEntry)) { + return false; + } + + CompactionLogEntry that = (CompactionLogEntry) o; + return dbSequenceNumber == that.dbSequenceNumber && + compactionTime == that.compactionTime && + Objects.equals(inputFileInfoList, that.inputFileInfoList) && + Objects.equals(outputFileInfoList, that.outputFileInfoList) && + Objects.equals(compactionReason, that.compactionReason); + } + + @Override + public int hashCode() { + return Objects.hash(dbSequenceNumber, compactionTime, inputFileInfoList, + outputFileInfoList, compactionReason); + } + + @Override + public CompactionLogEntry copyObject() { + return new CompactionLogEntry(dbSequenceNumber, compactionTime, + inputFileInfoList, outputFileInfoList, compactionReason); + } } diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java index 46fb8c179576..dd85c9d5ea3c 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java @@ -464,7 +464,7 @@ public void setSnapshotInfoTableCFHandle( * Set CompactionLogTable DB column family handle to be used in DB listener. * @param compactionLogTableCFHandle ColumnFamilyHandle */ - public void setCompactionLogTableCFHandle( + public synchronized void setCompactionLogTableCFHandle( ColumnFamilyHandle compactionLogTableCFHandle) { Preconditions.checkNotNull(compactionLogTableCFHandle, "Column family handle should not be null"); @@ -475,7 +475,7 @@ public void setCompactionLogTableCFHandle( * Set activeRocksDB to access CompactionLogTable. * @param activeRocksDB RocksDB */ - public void setActiveRocksDB(RocksDB activeRocksDB) { + public synchronized void setActiveRocksDB(RocksDB activeRocksDB) { Preconditions.checkNotNull(activeRocksDB, "RocksDB should not be null."); this.activeRocksDB = activeRocksDB; } @@ -1269,7 +1269,7 @@ public void pruneOlderSnapshotsWithCompactionHistory() { } Pair, List> fileNodeToKeyPair = - getOlderSnapshotFileNodes(); + getOlderFileNodes(); List lastCompactionSstFiles = fileNodeToKeyPair.getLeft(); List keysToRemove = fileNodeToKeyPair.getRight(); @@ -1289,7 +1289,11 @@ public void pruneOlderSnapshotsWithCompactionHistory() { } } - private Pair, List> getOlderSnapshotFileNodes() { + /** + * Returns the list of input files from the compaction entries which are + * older than the maximum allowed in the compaction DAG. + */ + private synchronized Pair, List> getOlderFileNodes() { long compactionLogPruneStartTime = System.currentTimeMillis(); List compactionNodes = new ArrayList<>(); List keysToRemove = new ArrayList<>(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java index 9ff97751308e..27055a96bc7a 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java @@ -109,6 +109,7 @@ import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.checkSnapshotDirExist; import org.apache.hadoop.util.Time; +import org.apache.ozone.rocksdiff.CompactionLogEntry; import org.apache.ratis.util.ExitUtils; import org.eclipse.jetty.util.StringUtil; import org.slf4j.Logger; @@ -280,6 +281,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager, private Table snapshotInfoTable; private Table snapshotRenamedTable; + private Table compactionLogTable; private boolean isRatisEnabled; private boolean ignorePipelineinKey; @@ -633,7 +635,8 @@ public static DBStoreBuilder addOMTablesAndCodecs(DBStoreBuilder builder) { .addCodec(OmDBTenantState.class, OmDBTenantState.getCodec()) .addCodec(OmDBAccessIdInfo.class, OmDBAccessIdInfo.getCodec()) .addCodec(OmDBUserPrincipalInfo.class, OmDBUserPrincipalInfo.getCodec()) - .addCodec(SnapshotInfo.class, SnapshotInfo.getCodec()); + .addCodec(SnapshotInfo.class, SnapshotInfo.getCodec()) + .addCodec(CompactionLogEntry.class, CompactionLogEntry.getCodec()); } /** @@ -746,6 +749,11 @@ protected void initializeOmTables(boolean addCacheMetrics) checkTableStatus(snapshotRenamedTable, SNAPSHOT_RENAMED_TABLE, addCacheMetrics); // TODO: [SNAPSHOT] Initialize table lock for snapshotRenamedTable. + + compactionLogTable = this.store.getTable(COMPACTION_LOG_TABLE, + String.class, String.class); + checkTableStatus(compactionLogTable, COMPACTION_LOG_TABLE, + addCacheMetrics); } /** diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java index 0d17faf1fb07..520893514840 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java @@ -43,6 +43,7 @@ import org.apache.hadoop.ozone.om.service.SnapshotDeletingService; import org.apache.hadoop.ozone.security.OzoneTokenIdentifier; import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos.PersistedUserVolumeInfo; +import org.apache.ozone.rocksdiff.CompactionLogEntry; import java.util.Map; @@ -231,6 +232,15 @@ public class OMDBDefinition extends DBDefinition.WithMap { SnapshotInfo.class, SnapshotInfo.getCodec()); + public static final DBColumnFamilyDefinition + COMPACTION_LOG_TABLE = + new DBColumnFamilyDefinition<>( + OmMetadataManagerImpl.COMPACTION_LOG_TABLE, + String.class, // snapshot path + StringCodec.get(), + CompactionLogEntry.class, + CompactionLogEntry.getCodec()); + /** * SnapshotRenamedTable that complements the keyTable (or fileTable) * and dirTable entries of the immediately previous snapshot in the @@ -268,6 +278,7 @@ public class OMDBDefinition extends DBDefinition.WithMap { S3_SECRET_TABLE, SNAPSHOT_INFO_TABLE, SNAPSHOT_RENAMED_TABLE, + COMPACTION_LOG_TABLE, TENANT_ACCESS_ID_TABLE, TENANT_STATE_TABLE, TRANSACTION_INFO_TABLE, From 422ae22998198bd22944e9d014ec3a5d31c9e993 Mon Sep 17 00:00:00 2001 From: Hemant Kumar Date: Thu, 14 Sep 2023 17:17:41 -0700 Subject: [PATCH 6/6] Fixed TestSnapshotBackgroundServices unit test failure --- .../log}/CompactionFileInfo.java | 2 +- .../log}/CompactionLogEntry.java | 2 +- .../ozone/compaction/log/package-info.java | 23 +++++ .../rocksdiff/RocksDBCheckpointDiffer.java | 14 ++- .../om/TestSnapshotBackgroundServices.java | 97 ++++++++----------- .../hadoop/ozone/om/OMMetadataManager.java | 2 + .../ozone/om/OmMetadataManagerImpl.java | 7 +- .../hadoop/ozone/om/codec/OMDBDefinition.java | 2 +- 8 files changed, 83 insertions(+), 66 deletions(-) rename hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/{rocksdiff => compaction/log}/CompactionFileInfo.java (98%) rename hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/{rocksdiff => compaction/log}/CompactionLogEntry.java (99%) create mode 100644 hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/compaction/log/package-info.java diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionFileInfo.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/compaction/log/CompactionFileInfo.java similarity index 98% rename from hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionFileInfo.java rename to hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/compaction/log/CompactionFileInfo.java index 90355797e7d8..15916b88078b 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionFileInfo.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/compaction/log/CompactionFileInfo.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.ozone.rocksdiff; +package org.apache.ozone.compaction.log; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/compaction/log/CompactionLogEntry.java similarity index 99% rename from hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java rename to hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/compaction/log/CompactionLogEntry.java index bc885bb8756b..a3393b78b15b 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionLogEntry.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/compaction/log/CompactionLogEntry.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.ozone.rocksdiff; +package org.apache.ozone.compaction.log; import org.apache.commons.collections.CollectionUtils; import org.apache.hadoop.hdds.StringUtils; diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/compaction/log/package-info.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/compaction/log/package-info.java new file mode 100644 index 000000000000..db93d8396626 --- /dev/null +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/compaction/log/package-info.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ozone.compaction.log; + +/** + * This package contains POJO classes for Compaction information. + */ diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java index dd85c9d5ea3c..0b61a44b1dc0 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java @@ -41,6 +41,8 @@ import org.apache.hadoop.hdds.utils.Scheduler; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB; +import org.apache.ozone.compaction.log.CompactionFileInfo; +import org.apache.ozone.compaction.log.CompactionLogEntry; import org.apache.ozone.rocksdb.util.RdbUtil; import org.apache.ozone.graph.PrintableGraph; import org.apache.ozone.graph.PrintableGraph.GraphType; @@ -166,7 +168,8 @@ public class RocksDBCheckpointDiffer implements AutoCloseable, * to save space. */ static final String SST_FILE_EXTENSION = ".sst"; - static final int SST_FILE_EXTENSION_LENGTH = SST_FILE_EXTENSION.length(); + public static final int SST_FILE_EXTENSION_LENGTH = + SST_FILE_EXTENSION.length(); private static final int LONG_MAX_STR_LEN = String.valueOf(Long.MAX_VALUE).length(); @@ -1165,12 +1168,12 @@ void dumpCompactionNodeTable() { } @VisibleForTesting - MutableGraph getForwardCompactionDAG() { + public MutableGraph getForwardCompactionDAG() { return forwardCompactionDAG; } @VisibleForTesting - MutableGraph getBackwardCompactionDAG() { + public MutableGraph getBackwardCompactionDAG() { return backwardCompactionDAG; } @@ -1518,11 +1521,6 @@ public static Logger getLog() { return LOG; } - @VisibleForTesting - public String getCurrentCompactionLogPath() { - return currentCompactionLogPath; - } - @VisibleForTesting public ConcurrentHashMap getCompactionNodeMap() { return compactionNodeMap; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java index 10df45b2680b..8d61fb1a3fa9 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java @@ -44,6 +44,8 @@ import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.snapshot.SnapshotDiffReportOzone; import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse; +import org.apache.ozone.compaction.log.CompactionLogEntry; +import org.apache.ozone.rocksdiff.CompactionNode; import org.apache.ozone.test.GenericTestUtils; import org.apache.ozone.test.LambdaTestUtils; import org.apache.ratis.server.protocol.TermIndex; @@ -55,12 +57,8 @@ import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.Timeout; -import java.io.BufferedReader; import java.io.File; import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -71,6 +69,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import static java.util.stream.Collectors.toSet; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL; @@ -404,36 +403,48 @@ public void testCompactionLogBackgroundService() cluster.getOzoneManager(leaderOM.getOMNodeId()); Assertions.assertEquals(leaderOM, newFollowerOM); - // Prepare baseline data for compaction logs - String currentCompactionLogPath = newLeaderOM - .getMetadataManager() - .getStore() - .getRocksDBCheckpointDiffer() - .getCurrentCompactionLogPath(); - Assertions.assertNotNull(currentCompactionLogPath); - int lastIndex = currentCompactionLogPath.lastIndexOf(OM_KEY_PREFIX); - String compactionLogsPath = currentCompactionLogPath - .substring(0, lastIndex); - File compactionLogsDir = new File(compactionLogsPath); - Assertions.assertNotNull(compactionLogsDir); - File[] files = compactionLogsDir.listFiles(); - Assertions.assertNotNull(files); - int numberOfLogFiles = files.length; - long contentLength; - Path currentCompactionLog = Paths.get(currentCompactionLogPath); - try (BufferedReader bufferedReader = - Files.newBufferedReader(currentCompactionLog)) { - contentLength = bufferedReader.lines() - .mapToLong(String::length) - .reduce(0L, Long::sum); - } + List compactionLogEntriesOnPreviousLeader = + getCompactionLogEntries(leaderOM); + + List compactionLogEntriesOnNewLeader = + getCompactionLogEntries(newLeaderOM); + Assertions.assertEquals(compactionLogEntriesOnPreviousLeader, + compactionLogEntriesOnNewLeader); + + Assertions.assertEquals(leaderOM.getMetadataManager().getStore() + .getRocksDBCheckpointDiffer().getForwardCompactionDAG().nodes() + .stream().map(CompactionNode::getFileName).collect(toSet()), + newLeaderOM.getMetadataManager().getStore() + .getRocksDBCheckpointDiffer().getForwardCompactionDAG().nodes() + .stream().map(CompactionNode::getFileName).collect(toSet())); + Assertions.assertEquals(leaderOM.getMetadataManager().getStore() + .getRocksDBCheckpointDiffer().getForwardCompactionDAG().edges() + .stream().map(edge -> + edge.source().getFileName() + "-" + edge.target().getFileName()) + .collect(toSet()), + newLeaderOM.getMetadataManager().getStore() + .getRocksDBCheckpointDiffer().getForwardCompactionDAG().edges() + .stream().map(edge -> + edge.source().getFileName() + "-" + edge.target().getFileName()) + .collect(toSet())); - checkIfCompactionLogsGetAppendedByForcingCompaction(newLeaderOM, - compactionLogsDir, numberOfLogFiles, contentLength, - currentCompactionLog); + confirmSnapDiffForTwoSnapshotsDifferingBySingleKey(newLeaderOM); + } - confirmSnapDiffForTwoSnapshotsDifferingBySingleKey( - newLeaderOM); + private List getCompactionLogEntries(OzoneManager om) + throws IOException { + List compactionLogEntries = new ArrayList<>(); + try (TableIterator> + iterator = om.getMetadataManager().getCompactionLogTable() + .iterator()) { + iterator.seekToFirst(); + + while (iterator.hasNext()) { + compactionLogEntries.add(iterator.next().getValue()); + } + } + return compactionLogEntries; } @Test @@ -546,28 +557,6 @@ private static void checkIfCompactionBackupFilesWerePruned( }, 1000, 10000); } - private static void checkIfCompactionLogsGetAppendedByForcingCompaction( - OzoneManager ozoneManager, - File compactionLogsDir, int numberOfLogFiles, - long contentLength, Path currentCompactionLog) - throws IOException { - ozoneManager.getMetadataManager() - .getStore() - .compactDB(); - File[] files = compactionLogsDir.listFiles(); - Assertions.assertNotNull(files); - int newNumberOfLogFiles = files.length; - long newContentLength; - try (BufferedReader bufferedReader = - Files.newBufferedReader(currentCompactionLog)) { - newContentLength = bufferedReader.lines() - .mapToLong(String::length) - .reduce(0L, Long::sum); - } - Assertions.assertTrue(numberOfLogFiles < newNumberOfLogFiles - || contentLength < newContentLength); - } - private static File getSstBackupDir(OzoneManager ozoneManager) { String sstBackupDirPath = ozoneManager .getMetadataManager() diff --git a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java index 3dcabf088e6d..0832344dcedd 100644 --- a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java +++ b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java @@ -51,6 +51,7 @@ import org.apache.hadoop.hdds.utils.db.Table; import com.google.common.annotations.VisibleForTesting; +import org.apache.ozone.compaction.log.CompactionLogEntry; /** * OM metadata manager interface. @@ -378,6 +379,7 @@ String getMultipartKey(String volume, String bucket, String key, String Table getSnapshotRenamedTable(); + Table getCompactionLogTable(); /** * Gets the OM Meta table. * @return meta table reference. diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java index 27055a96bc7a..b1cb80645fe2 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java @@ -109,7 +109,7 @@ import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.checkSnapshotDirExist; import org.apache.hadoop.util.Time; -import org.apache.ozone.rocksdiff.CompactionLogEntry; +import org.apache.ozone.compaction.log.CompactionLogEntry; import org.apache.ratis.util.ExitUtils; import org.eclipse.jetty.util.StringUtil; import org.slf4j.Logger; @@ -1954,6 +1954,11 @@ public Table getSnapshotRenamedTable() { return snapshotRenamedTable; } + @Override + public Table getCompactionLogTable() { + return compactionLogTable; + } + /** * Get Snapshot Chain Manager. * diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java index 520893514840..a70d05ab636c 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java @@ -43,7 +43,7 @@ import org.apache.hadoop.ozone.om.service.SnapshotDeletingService; import org.apache.hadoop.ozone.security.OzoneTokenIdentifier; import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos.PersistedUserVolumeInfo; -import org.apache.ozone.rocksdiff.CompactionLogEntry; +import org.apache.ozone.compaction.log.CompactionLogEntry; import java.util.Map;