aswinshakil commented on code in PR #8053: URL: https://github.com/apache/ozone/pull/8053#discussion_r2046265034
########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableFilter.java: ########## @@ -0,0 +1,247 @@ +/* + * 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.hadoop.ozone.om.snapshot.filter; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import org.apache.hadoop.hdds.utils.IOUtils; +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.ozone.om.KeyManager; +import org.apache.hadoop.ozone.om.OmSnapshot; +import org.apache.hadoop.ozone.om.OmSnapshotManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.SnapshotChainManager; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; +import org.apache.hadoop.ozone.om.lock.OzoneManagerLock; +import org.apache.hadoop.ozone.om.snapshot.MultiSnapshotLocks; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; +import org.apache.hadoop.ozone.util.CheckedFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class is responsible for opening last N snapshot given a snapshot metadata manager or AOS metadata manager by + * acquiring a lock. + */ +public abstract class ReclaimableFilter<V> implements CheckedFunction<Table.KeyValue<String, V>, + Boolean, IOException>, Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(ReclaimableFilter.class); + + private final OzoneManager ozoneManager; + private final SnapshotInfo currentSnapshotInfo; + private final OmSnapshotManager omSnapshotManager; + private final SnapshotChainManager snapshotChainManager; + private final List<SnapshotInfo> tmpValidationSnapshotInfos; + private final List<UUID> lockedSnapshotIds; + private final List<SnapshotInfo> previousSnapshotInfos; + private final List<ReferenceCounted<OmSnapshot>> previousOmSnapshots; + private final MultiSnapshotLocks snapshotIdLocks; + private Long volumeId; + private OmBucketInfo bucketInfo; + private final KeyManager keyManager; + private final int numberOfPreviousSnapshotsFromChain; + + /** + * Filter to return deleted keys/directories which are reclaimable based on their presence in previous snapshot in + * the snapshot chain. + * + * @param ozoneManager : Ozone Manager instance + * @param omSnapshotManager : OmSnapshot Manager of OM instance. + * @param snapshotChainManager : snapshot chain manager of OM instance. + * @param currentSnapshotInfo : If null the deleted keys in Active Metadata manager needs to be processed, hence the + * latest snapshot in the snapshot chain corresponding to bucket key needs to be + * processed. + * @param keyManager : KeyManager corresponding to snapshot or Active Metadata Manager. + * @param lock : Lock for Active OM. + * @param numberOfPreviousSnapshotsFromChain : number of previous snapshots to be initialized. + */ + public ReclaimableFilter(OzoneManager ozoneManager, OmSnapshotManager omSnapshotManager, + SnapshotChainManager snapshotChainManager, + SnapshotInfo currentSnapshotInfo, KeyManager keyManager, + IOzoneManagerLock lock, + int numberOfPreviousSnapshotsFromChain) { + this.ozoneManager = ozoneManager; + this.omSnapshotManager = omSnapshotManager; + this.currentSnapshotInfo = currentSnapshotInfo; + this.snapshotChainManager = snapshotChainManager; + this.snapshotIdLocks = new MultiSnapshotLocks(lock, OzoneManagerLock.Resource.SNAPSHOT_GC_LOCK, false); + this.keyManager = keyManager; + this.numberOfPreviousSnapshotsFromChain = numberOfPreviousSnapshotsFromChain; + this.previousOmSnapshots = new ArrayList<>(numberOfPreviousSnapshotsFromChain); + this.previousSnapshotInfos = new ArrayList<>(numberOfPreviousSnapshotsFromChain); + // Used for tmp list to avoid lots of garbage collection of list. + this.tmpValidationSnapshotInfos = new ArrayList<>(numberOfPreviousSnapshotsFromChain); + this.lockedSnapshotIds = new ArrayList<>(numberOfPreviousSnapshotsFromChain + 1); + } + + private List<SnapshotInfo> getLastNSnapshotInChain(String volume, String bucket) throws IOException { + if (currentSnapshotInfo != null && + (!currentSnapshotInfo.getVolumeName().equals(volume) || !currentSnapshotInfo.getBucketName().equals(bucket))) { + throw new IOException("Volume and Bucket name for snapshot : " + currentSnapshotInfo + " do not match " + + "against the volume: " + volume + " and bucket: " + bucket + " of the key."); + } + tmpValidationSnapshotInfos.clear(); + SnapshotInfo expectedPreviousSnapshotInfo = currentSnapshotInfo == null + ? SnapshotUtils.getLatestSnapshotInfo(volume, bucket, ozoneManager, snapshotChainManager) + : SnapshotUtils.getPreviousSnapshot(ozoneManager, snapshotChainManager, currentSnapshotInfo); + SnapshotInfo snapshotInfo = expectedPreviousSnapshotInfo; + while (tmpValidationSnapshotInfos.size() < numberOfPreviousSnapshotsFromChain) { Review Comment: If `snapshotInfo` is `null` there is no previous snapshot, Can we terminate this loop early? ```suggestion while (snapshotInfo != null && tmpValidationSnapshotInfos.size() < numberOfPreviousSnapshotsFromChain) { ``` ########## hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/filter/TestReclaimableFilter.java: ########## @@ -0,0 +1,288 @@ +/* + * 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.hadoop.ozone.om.snapshot.filter; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.hadoop.hdds.utils.TransactionInfo; +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.ozone.om.KeyManager; +import org.apache.hadoop.ozone.om.OmSnapshot; +import org.apache.hadoop.ozone.om.OmSnapshotManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.SnapshotChainManager; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.rocksdb.RocksDBException; + +/** + * Test class for ReclaimableFilter testing general initializing of snapshot chain. + */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class TestReclaimableFilter extends TestAbstractReclaimableFilter { + + @Override + protected ReclaimableFilter initializeFilter(OzoneManager om, OmSnapshotManager snapshotManager, + SnapshotChainManager chainManager, + SnapshotInfo currentSnapshotInfo, KeyManager km, + IOzoneManagerLock lock, int numberOfPreviousSnapshotsFromChain) { + return new ReclaimableFilter<Boolean>(om, snapshotManager, chainManager, currentSnapshotInfo, + km, lock, numberOfPreviousSnapshotsFromChain) { + @Override + protected String getVolumeName(Table.KeyValue<String, Boolean> keyValue) throws IOException { + return keyValue.getKey().split("/")[0]; + } + + @Override + protected String getBucketName(Table.KeyValue<String, Boolean> keyValue) throws IOException { + return keyValue.getKey().split("/")[1]; + } + + @Override + protected Boolean isReclaimable(Table.KeyValue<String, Boolean> keyValue) throws IOException { + return keyValue == null || keyValue.getValue(); + } + }; + } + + List<Arguments> testReclaimableFilterArguments() { Review Comment: We should add a doc to specify what these parameters are. ########## hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/filter/TestReclaimableFilter.java: ########## @@ -0,0 +1,288 @@ +/* + * 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.hadoop.ozone.om.snapshot.filter; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.hadoop.hdds.utils.TransactionInfo; +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.ozone.om.KeyManager; +import org.apache.hadoop.ozone.om.OmSnapshot; +import org.apache.hadoop.ozone.om.OmSnapshotManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.SnapshotChainManager; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.rocksdb.RocksDBException; + +/** + * Test class for ReclaimableFilter testing general initializing of snapshot chain. + */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class TestReclaimableFilter extends TestAbstractReclaimableFilter { + + @Override + protected ReclaimableFilter initializeFilter(OzoneManager om, OmSnapshotManager snapshotManager, + SnapshotChainManager chainManager, + SnapshotInfo currentSnapshotInfo, KeyManager km, + IOzoneManagerLock lock, int numberOfPreviousSnapshotsFromChain) { + return new ReclaimableFilter<Boolean>(om, snapshotManager, chainManager, currentSnapshotInfo, + km, lock, numberOfPreviousSnapshotsFromChain) { + @Override + protected String getVolumeName(Table.KeyValue<String, Boolean> keyValue) throws IOException { + return keyValue.getKey().split("/")[0]; + } + + @Override + protected String getBucketName(Table.KeyValue<String, Boolean> keyValue) throws IOException { + return keyValue.getKey().split("/")[1]; + } + + @Override + protected Boolean isReclaimable(Table.KeyValue<String, Boolean> keyValue) throws IOException { + return keyValue == null || keyValue.getValue(); + } + }; + } + + List<Arguments> testReclaimableFilterArguments() { + List<Arguments> arguments = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + for (int k = 0; k < 5; k++) { + arguments.add(Arguments.of(i, j, k)); + } + } + } + return arguments; + } + + private void testSnapshotInitAndLocking(String volume, String bucket, int numberOfPreviousSnapshotsFromChain, + int index, SnapshotInfo currentSnapshotInfo, Boolean reclaimable, + Boolean expectedReturnValue) throws IOException { + List<SnapshotInfo> infos = getLastSnapshotInfos(volume, bucket, numberOfPreviousSnapshotsFromChain, index); + assertEquals(expectedReturnValue, + getReclaimableFilter().apply(Table.newKeyValue(getKey(volume, bucket), reclaimable))); + Assertions.assertEquals(infos, getReclaimableFilter().getPreviousSnapshotInfos()); + Assertions.assertEquals(infos.size(), getReclaimableFilter().getPreviousOmSnapshots().size()); + Assertions.assertEquals(infos.stream().map(si -> si == null ? null : si.getSnapshotId()) + .collect(Collectors.toList()), getReclaimableFilter().getPreviousOmSnapshots().stream() + .map(i -> i == null ? null : ((ReferenceCounted<OmSnapshot>) i).get().getSnapshotID()) + .collect(Collectors.toList())); + infos.add(currentSnapshotInfo); + Assertions.assertEquals(infos.stream().filter(Objects::nonNull).map(SnapshotInfo::getSnapshotId).collect( + Collectors.toList()), getLockIds().get()); + } + + @ParameterizedTest + @MethodSource("testReclaimableFilterArguments") + public void testReclaimableFilterSnapshotChainInitilization(int numberOfPreviousSnapshotsFromChain, Review Comment: ```suggestion public void testReclaimableFilterSnapshotChainInitialization(int numberOfPreviousSnapshotsFromChain, ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
