diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java index 09cbf4fe4e40..34dcba40f81b 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java @@ -178,9 +178,7 @@ public static BucketHandler getBucketHandler( omMetadataManager, reconSCM, bucketInfo); } else if (bucketInfo.getBucketLayout() .equals(BucketLayout.OBJECT_STORE)) { - // TODO: HDDS-7810 Write a handler for object store bucket - // We can use LegacyBucketHandler for OBS bucket for now. - return new LegacyBucketHandler(reconNamespaceSummaryManager, + return new OBSBucketHandler(reconNamespaceSummaryManager, omMetadataManager, reconSCM, bucketInfo); } else { LOG.error("Unsupported bucket layout: " + diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/FSOBucketHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/FSOBucketHandler.java index 26cda6442d4e..8a1c5babe75e 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/FSOBucketHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/FSOBucketHandler.java @@ -42,7 +42,7 @@ import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; /** - * Class for handling FSO buckets. + * Class for handling FSO buckets NameSpaceSummaries. */ public class FSOBucketHandler extends BucketHandler { private static final Logger LOG = diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/LegacyBucketHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/LegacyBucketHandler.java index 3dd1ddbdabb9..09f1c5bc7454 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/LegacyBucketHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/LegacyBucketHandler.java @@ -41,7 +41,7 @@ import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; /** - * Class for handling Legacy buckets. + * Class for handling Legacy buckets NameSpaceSummaries. */ public class LegacyBucketHandler extends BucketHandler { diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/OBSBucketHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/OBSBucketHandler.java new file mode 100644 index 000000000000..024eec989a10 --- /dev/null +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/OBSBucketHandler.java @@ -0,0 +1,268 @@ +/* + * 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.recon.api.handlers; + + +import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager; +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.hdds.utils.db.TableIterator; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.recon.api.types.DUResponse; +import org.apache.hadoop.ozone.recon.api.types.EntityType; +import org.apache.hadoop.ozone.recon.api.types.NSSummary; +import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; +import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; + +import java.io.IOException; +import java.util.List; + +import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; + +/** + * Class for handling OBS buckets NameSpaceSummaries. + */ +public class OBSBucketHandler extends BucketHandler { + + private final String vol; + private final String bucket; + private final OmBucketInfo omBucketInfo; + + public OBSBucketHandler( + ReconNamespaceSummaryManager reconNamespaceSummaryManager, + ReconOMMetadataManager omMetadataManager, + OzoneStorageContainerManager reconSCM, + OmBucketInfo bucketInfo) { + super(reconNamespaceSummaryManager, omMetadataManager, + reconSCM); + this.omBucketInfo = bucketInfo; + this.vol = omBucketInfo.getVolumeName(); + this.bucket = omBucketInfo.getBucketName(); + } + + /** + * Helper function to check if a path is a key, or invalid. + * + * @param keyName key name + * @return KEY, or UNKNOWN + * @throws IOException + */ + @Override + public EntityType determineKeyPath(String keyName) throws IOException { + String key = OM_KEY_PREFIX + vol + + OM_KEY_PREFIX + bucket + + OM_KEY_PREFIX + keyName; + + Table keyTable = getKeyTable(); + + try ( + TableIterator> + iterator = keyTable.iterator()) { + iterator.seek(key); + if (iterator.hasNext()) { + Table.KeyValue kv = iterator.next(); + String dbKey = kv.getKey(); + if (dbKey.equals(key)) { + return EntityType.KEY; + } + } + } + return EntityType.UNKNOWN; + } + + /** + * This method handles disk usage of direct keys. + * + * @param parentId The identifier for the parent bucket. + * @param withReplica if withReplica is enabled, set sizeWithReplica + * for each direct key's DU + * @param listFile if listFile is enabled, append key DU as a children + * keys + * @param duData the current DU data + * @param normalizedPath the normalized path request + * @return the total DU of all direct keys + * @throws IOException IOE + */ + @Override + public long handleDirectKeys(long parentId, boolean withReplica, + boolean listFile, + List duData, + String normalizedPath) throws IOException { + + NSSummary nsSummary = getReconNamespaceSummaryManager() + .getNSSummary(parentId); + // Handle the case of an empty bucket. + if (nsSummary == null) { + return 0; + } + + Table keyTable = getKeyTable(); + long keyDataSizeWithReplica = 0L; + + try ( + TableIterator> + iterator = keyTable.iterator()) { + + String seekPrefix = OM_KEY_PREFIX + + vol + + OM_KEY_PREFIX + + bucket + + OM_KEY_PREFIX; + + iterator.seek(seekPrefix); + + while (iterator.hasNext()) { + // KeyName : OmKeyInfo-Object + Table.KeyValue kv = iterator.next(); + String dbKey = kv.getKey(); + + // Exit loop if the key doesn't match the seekPrefix. + if (!dbKey.startsWith(seekPrefix)) { + break; + } + + OmKeyInfo keyInfo = kv.getValue(); + if (keyInfo != null) { + DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage(); + String objectName = keyInfo.getKeyName(); + diskUsage.setSubpath(objectName); + diskUsage.setKey(true); + diskUsage.setSize(keyInfo.getDataSize()); + + if (withReplica) { + long keyDU = keyInfo.getReplicatedSize(); + keyDataSizeWithReplica += keyDU; + diskUsage.setSizeWithReplica(keyDU); + } + // List all the keys for the OBS bucket if requested. + if (listFile) { + duData.add(diskUsage); + } + } + } + } + + return keyDataSizeWithReplica; + } + + /** + * Calculates the total disk usage (DU) for an Object Store Bucket (OBS) by + * summing the sizes of all keys contained within the bucket. + * Since OBS buckets operate on a flat hierarchy, this method iterates through + * all the keys in the bucket without the need to traverse directories. + * + * @param parentId The identifier for the parent bucket. + * @return The total disk usage of all keys within the specified OBS bucket. + * @throws IOException + */ + @Override + public long calculateDUUnderObject(long parentId) throws IOException { + // Initialize the total disk usage variable. + long totalDU = 0L; + + // Access the key table for the bucket. + Table keyTable = getKeyTable(); + + try ( + TableIterator> + iterator = keyTable.iterator()) { + // Construct the seek prefix to filter keys under this bucket. + String seekPrefix = + OM_KEY_PREFIX + vol + OM_KEY_PREFIX + bucket + OM_KEY_PREFIX; + iterator.seek(seekPrefix); + + // Iterate over keys in the bucket. + while (iterator.hasNext()) { + Table.KeyValue kv = iterator.next(); + String keyName = kv.getKey(); + + // Break the loop if the current key does not start with the seekPrefix. + if (!keyName.startsWith(seekPrefix)) { + break; + } + + // Sum the size of each key to the total disk usage. + OmKeyInfo keyInfo = kv.getValue(); + if (keyInfo != null) { + totalDU += keyInfo.getDataSize(); + } + } + } + + // Return the total disk usage of all keys in the bucket. + return totalDU; + } + + /** + * Object stores do not support directories. + * + * @throws UnsupportedOperationException + */ + @Override + public long getDirObjectId(String[] names) + throws UnsupportedOperationException { + throw new UnsupportedOperationException( + "Object stores do not support directories."); + } + + /** + * Object stores do not support directories. + * + * @throws UnsupportedOperationException + */ + @Override + public long getDirObjectId(String[] names, int cutoff) + throws UnsupportedOperationException { + throw new UnsupportedOperationException( + "Object stores do not support directories."); + } + + /** + * Returns the keyInfo object from the KEY table. + * @return OmKeyInfo + */ + @Override + public OmKeyInfo getKeyInfo(String[] names) throws IOException { + String ozoneKey = OM_KEY_PREFIX; + ozoneKey += String.join(OM_KEY_PREFIX, names); + + return getKeyTable().getSkipCache(ozoneKey); + } + + /** + * Object stores do not support directories. + * + * @throws UnsupportedOperationException + */ + @Override + public OmDirectoryInfo getDirInfo(String[] names) throws IOException { + throw new UnsupportedOperationException( + "Object stores do not support directories."); + } + + public Table getKeyTable() { + return getOmMetadataManager().getKeyTable(getBucketLayout()); + } + + public BucketLayout getBucketLayout() { + return BucketLayout.OBJECT_STORE; + } + +} diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java index 42356191c501..5c3395084464 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java @@ -38,6 +38,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.Future; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; /** * Task to query data from OMDB and write into Recon RocksDB. @@ -68,6 +69,7 @@ public class NSSummaryTask implements ReconOmTask { private final ReconOMMetadataManager reconOMMetadataManager; private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO; private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy; + private final NSSummaryTaskWithOBS nsSummaryTaskWithOBS; private final OzoneConfiguration ozoneConfiguration; @Inject @@ -86,6 +88,9 @@ public NSSummaryTask(ReconNamespaceSummaryManager this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy( reconNamespaceSummaryManager, reconOMMetadataManager, ozoneConfiguration); + this.nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( + reconNamespaceSummaryManager, + reconOMMetadataManager, ozoneConfiguration); } @Override @@ -95,20 +100,28 @@ public String getTaskName() { @Override public Pair process(OMUpdateEventBatch events) { - boolean success; - success = nsSummaryTaskWithFSO.processWithFSO(events); - if (success) { - success = nsSummaryTaskWithLegacy.processWithLegacy(events); - } else { + boolean success = nsSummaryTaskWithFSO.processWithFSO(events); + if (!success) { LOG.error("processWithFSO failed."); } + success = nsSummaryTaskWithLegacy.processWithLegacy(events); + if (!success) { + LOG.error("processWithLegacy failed."); + } + success = nsSummaryTaskWithOBS.processWithOBS(events); + if (!success) { + LOG.error("processWithOBS failed."); + } return new ImmutablePair<>(getTaskName(), success); } @Override public Pair reprocess(OMMetadataManager omMetadataManager) { + // Initialize a list of tasks to run in parallel Collection> tasks = new ArrayList<>(); + long startTime = System.nanoTime(); // Record start time + try { // reinit Recon RocksDB's namespace CF. reconNamespaceSummaryManager.clearNSSummaryTable(); @@ -122,6 +135,8 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { .reprocessWithFSO(omMetadataManager)); tasks.add(() -> nsSummaryTaskWithLegacy .reprocessWithLegacy(reconOMMetadataManager)); + tasks.add(() -> nsSummaryTaskWithOBS + .reprocessWithOBS(reconOMMetadataManager)); List> results; ThreadFactory threadFactory = new ThreadFactoryBuilder() @@ -137,17 +152,25 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { } } } catch (InterruptedException ex) { - LOG.error("Error while reprocessing NSSummary " + - "table in Recon DB. ", ex); + LOG.error("Error while reprocessing NSSummary table in Recon DB.", ex); return new ImmutablePair<>(getTaskName(), false); } catch (ExecutionException ex2) { - LOG.error("Error while reprocessing NSSummary " + - "table in Recon DB. ", ex2); + LOG.error("Error while reprocessing NSSummary table in Recon DB.", ex2); return new ImmutablePair<>(getTaskName(), false); } finally { executorService.shutdown(); + + long endTime = System.nanoTime(); + // Convert to milliseconds + long durationInMillis = + TimeUnit.NANOSECONDS.toMillis(endTime - startTime); + + // Log performance metrics + LOG.info("Task execution time: {} milliseconds", durationInMillis); } + return new ImmutablePair<>(getTaskName(), true); } + } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java new file mode 100644 index 000000000000..34c7dc967c3a --- /dev/null +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java @@ -0,0 +1,236 @@ +/* + * 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.recon.tasks; + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.hdds.utils.db.TableIterator; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.WithParentObjectId; +import org.apache.hadoop.ozone.recon.api.types.NSSummary; +import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; +import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE; + + +/** + * Class for handling OBS specific tasks. + */ +public class NSSummaryTaskWithOBS extends NSSummaryTaskDbEventHandler { + + private static final BucketLayout BUCKET_LAYOUT = BucketLayout.OBJECT_STORE; + + private static final Logger LOG = + LoggerFactory.getLogger(NSSummaryTaskWithOBS.class); + + + public NSSummaryTaskWithOBS( + ReconNamespaceSummaryManager reconNamespaceSummaryManager, + ReconOMMetadataManager reconOMMetadataManager, + OzoneConfiguration ozoneConfiguration) { + super(reconNamespaceSummaryManager, + reconOMMetadataManager, ozoneConfiguration); + } + + + public boolean reprocessWithOBS(OMMetadataManager omMetadataManager) { + Map nsSummaryMap = new HashMap<>(); + + try { + Table keyTable = + omMetadataManager.getKeyTable(BUCKET_LAYOUT); + + try (TableIterator> + keyTableIter = keyTable.iterator()) { + + while (keyTableIter.hasNext()) { + Table.KeyValue kv = keyTableIter.next(); + OmKeyInfo keyInfo = kv.getValue(); + + // KeyTable entries belong to both Legacy and OBS buckets. + // Check bucket layout and if it's anything other than OBS, + // continue to the next iteration. + String volumeName = keyInfo.getVolumeName(); + String bucketName = keyInfo.getBucketName(); + String bucketDBKey = omMetadataManager + .getBucketKey(volumeName, bucketName); + // Get bucket info from bucket table + OmBucketInfo omBucketInfo = omMetadataManager + .getBucketTable().getSkipCache(bucketDBKey); + + if (omBucketInfo.getBucketLayout() != BUCKET_LAYOUT) { + continue; + } + + setKeyParentID(keyInfo); + + handlePutKeyEvent(keyInfo, nsSummaryMap); + if (!checkAndCallFlushToDB(nsSummaryMap)) { + return false; + } + } + } + } catch (IOException ioEx) { + LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", + ioEx); + return false; + } + + // flush and commit left out entries at end + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return false; + } + LOG.info("Completed a reprocess run of NSSummaryTaskWithOBS"); + return true; + } + + public boolean processWithOBS(OMUpdateEventBatch events) { + Iterator eventIterator = events.getIterator(); + Map nsSummaryMap = new HashMap<>(); + + while (eventIterator.hasNext()) { + OMDBUpdateEvent omdbUpdateEvent = + eventIterator.next(); + OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction(); + + // We only process updates on OM's KeyTable + String table = omdbUpdateEvent.getTable(); + boolean updateOnKeyTable = table.equals(KEY_TABLE); + if (!updateOnKeyTable) { + continue; + } + + String updatedKey = omdbUpdateEvent.getKey(); + + try { + OMDBUpdateEvent keyTableUpdateEvent = omdbUpdateEvent; + Object value = keyTableUpdateEvent.getValue(); + Object oldValue = keyTableUpdateEvent.getOldValue(); + if (value == null) { + LOG.warn("Value is null for key {}. Skipping processing.", + updatedKey); + continue; + } else if (!(value instanceof OmKeyInfo)) { + LOG.warn("Unexpected value type {} for key {}. Skipping processing.", + value.getClass().getName(), updatedKey); + continue; + } + + OmKeyInfo updatedKeyInfo = (OmKeyInfo) value; + OmKeyInfo oldKeyInfo = (OmKeyInfo) oldValue; + + // KeyTable entries belong to both OBS and Legacy buckets. + // Check bucket layout and if it's anything other than OBS, + // continue to the next iteration. + String volumeName = updatedKeyInfo.getVolumeName(); + String bucketName = updatedKeyInfo.getBucketName(); + String bucketDBKey = + getReconOMMetadataManager().getBucketKey(volumeName, bucketName); + // Get bucket info from bucket table + OmBucketInfo omBucketInfo = getReconOMMetadataManager().getBucketTable() + .getSkipCache(bucketDBKey); + + if (omBucketInfo.getBucketLayout() != BUCKET_LAYOUT) { + continue; + } + + setKeyParentID(updatedKeyInfo); + + switch (action) { + case PUT: + handlePutKeyEvent(updatedKeyInfo, nsSummaryMap); + break; + case DELETE: + handleDeleteKeyEvent(updatedKeyInfo, nsSummaryMap); + break; + case UPDATE: + if (oldKeyInfo != null) { + // delete first, then put + setKeyParentID(oldKeyInfo); + handleDeleteKeyEvent(oldKeyInfo, nsSummaryMap); + } else { + LOG.warn("Update event does not have the old keyInfo for {}.", + updatedKey); + } + handlePutKeyEvent(updatedKeyInfo, nsSummaryMap); + break; + default: + LOG.debug("Skipping DB update event: {}", action); + } + + if (!checkAndCallFlushToDB(nsSummaryMap)) { + return false; + } + } catch (IOException ioEx) { + LOG.error("Unable to process Namespace Summary data in Recon DB. ", + ioEx); + return false; + } + if (!checkAndCallFlushToDB(nsSummaryMap)) { + return false; + } + } + + // Flush and commit left-out entries at the end + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return false; + } + + LOG.info("Completed a process run of NSSummaryTaskWithOBS"); + return true; + } + + + /** + * KeyTable entries don't have the parentId set. + * In order to reuse the existing methods that rely on + * the parentId, we have to set it explicitly. + * Note: For an OBS key, the parentId will always correspond to the ID of the + * OBS bucket in which it is located. + * + * @param keyInfo + * @throws IOException + */ + private void setKeyParentID(OmKeyInfo keyInfo) + throws IOException { + String bucketKey = getReconOMMetadataManager() + .getBucketKey(keyInfo.getVolumeName(), keyInfo.getBucketName()); + OmBucketInfo parentBucketInfo = + getReconOMMetadataManager().getBucketTable().getSkipCache(bucketKey); + + if (parentBucketInfo != null) { + keyInfo.setParentObjectID(parentBucketInfo.getObjectID()); + } else { + throw new IOException("ParentKeyInfo for " + + "NSSummaryTaskWithOBS is null"); + } + } + +} diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBS.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBS.java new file mode 100644 index 000000000000..ac8dee5f0937 --- /dev/null +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBS.java @@ -0,0 +1,1127 @@ +/* + * 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.recon.api; + +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE; +import static org.apache.hadoop.ozone.om.helpers.QuotaUtil.getReplicatedSize; + +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.container.ContainerManager; +import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; +import org.apache.hadoop.hdds.scm.container.ContainerReplica; +import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat; +import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.om.OMConfigKeys; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; +import org.apache.hadoop.ozone.recon.ReconConstants; +import org.apache.hadoop.ozone.recon.ReconTestInjector; +import org.apache.hadoop.ozone.recon.api.handlers.BucketHandler; +import org.apache.hadoop.ozone.recon.api.handlers.EntityHandler; +import org.apache.hadoop.ozone.recon.api.types.BucketObjectDBInfo; +import org.apache.hadoop.ozone.recon.api.types.DUResponse; +import org.apache.hadoop.ozone.recon.api.types.EntityType; +import org.apache.hadoop.ozone.recon.api.types.NamespaceSummaryResponse; +import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse; +import org.apache.hadoop.ozone.recon.api.types.ResponseStatus; +import org.apache.hadoop.ozone.recon.api.types.VolumeObjectDBInfo; +import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse; +import org.apache.hadoop.ozone.recon.common.CommonUtils; +import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; +import org.apache.hadoop.ozone.recon.scm.ReconNodeManager; +import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade; +import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; +import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider; +import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl; +import org.apache.hadoop.ozone.recon.spi.impl.StorageContainerServiceProviderImpl; +import org.apache.hadoop.ozone.recon.tasks.NSSummaryTaskWithOBS; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import javax.ws.rs.core.Response; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.ArrayList; +import java.util.Set; +import java.util.HashSet; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_DB_DIRS; +import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm; +import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProvider; +import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Test for NSSummary REST APIs with OBS. + * Testing is done on a simple object store model with a flat hierarchy: + * Testing the following case. + * ├── vol + * │ ├── bucket1 + * │ │ ├── file1 + * │ │ └── file2 + * │ │ └── file3 + * │ └── bucket2 + * │ ├── file4 + * │ └── file5 + * └── vol2 + * ├── bucket3 + * │ ├── file8 + * │ ├── file9 + * │ └── file10 + * └── bucket4 + * └── file11 + * This tests the Rest APIs for NSSummary in the context of OBS buckets, + * focusing on disk usage, quota usage, and file size distribution. + */ +public class TestNSSummaryEndpointWithOBS { + @TempDir + private Path temporaryFolder; + + private ReconOMMetadataManager reconOMMetadataManager; + private NSSummaryEndpoint nsSummaryEndpoint; + private OzoneConfiguration conf; + private CommonUtils commonUtils; + + private static final String TEST_PATH_UTILITY = + "/vol1/buck1/a/b/c/d/e/file1.txt"; + private static final String PARENT_DIR = "vol1/buck1/a/b/c/d/e"; + private static final String[] TEST_NAMES = + new String[]{"vol1", "buck1", "a", "b", "c", "d", "e", "file1.txt"}; + private static final String TEST_KEY_NAMES = "a/b/c/d/e/file1.txt"; + + // Object names + private static final String VOL = "vol"; + private static final String VOL_TWO = "vol2"; + private static final String BUCKET_ONE = "bucket1"; + private static final String BUCKET_TWO = "bucket2"; + private static final String BUCKET_THREE = "bucket3"; + private static final String BUCKET_FOUR = "bucket4"; + private static final String KEY_ONE = "file1"; + private static final String KEY_TWO = "file2"; + private static final String KEY_THREE = "file3"; + private static final String KEY_FOUR = "file4"; + private static final String KEY_FIVE = "file5"; + private static final String KEY_EIGHT = "file8"; + private static final String KEY_NINE = "file9"; + private static final String KEY_TEN = "file10"; + private static final String KEY_ELEVEN = "file11"; + private static final String MULTI_BLOCK_FILE = KEY_THREE; + + private static final long PARENT_OBJECT_ID_ZERO = 0L; + private static final long VOL_OBJECT_ID = 0L; + private static final long VOL_TWO_OBJECT_ID = 14L; + private static final long BUCKET_ONE_OBJECT_ID = 1L; + private static final long BUCKET_TWO_OBJECT_ID = 2L; + private static final long BUCKET_THREE_OBJECT_ID = 15L; + private static final long BUCKET_FOUR_OBJECT_ID = 16L; + private static final long KEY_ONE_OBJECT_ID = 3L; + private static final long KEY_TWO_OBJECT_ID = 5L; + private static final long KEY_THREE_OBJECT_ID = 8L; + private static final long KEY_FOUR_OBJECT_ID = 6L; + private static final long KEY_FIVE_OBJECT_ID = 9L; + private static final long KEY_EIGHT_OBJECT_ID = 17L; + private static final long KEY_NINE_OBJECT_ID = 19L; + private static final long KEY_TEN_OBJECT_ID = 20L; + private static final long KEY_ELEVEN_OBJECT_ID = 21L; + private static final long MULTI_BLOCK_KEY_OBJECT_ID = 13L; + + // container IDs + private static final long CONTAINER_ONE_ID = 1L; + private static final long CONTAINER_TWO_ID = 2L; + private static final long CONTAINER_THREE_ID = 3L; + private static final long CONTAINER_FOUR_ID = 4L; + private static final long CONTAINER_FIVE_ID = 5L; + private static final long CONTAINER_SIX_ID = 6L; + + // replication factors + private static final int CONTAINER_ONE_REPLICA_COUNT = 3; + private static final int CONTAINER_TWO_REPLICA_COUNT = 2; + private static final int CONTAINER_THREE_REPLICA_COUNT = 4; + private static final int CONTAINER_FOUR_REPLICA_COUNT = 5; + private static final int CONTAINER_FIVE_REPLICA_COUNT = 2; + private static final int CONTAINER_SIX_REPLICA_COUNT = 3; + + // block lengths + private static final long BLOCK_ONE_LENGTH = 1000L; + private static final long BLOCK_TWO_LENGTH = 2000L; + private static final long BLOCK_THREE_LENGTH = 3000L; + private static final long BLOCK_FOUR_LENGTH = 4000L; + private static final long BLOCK_FIVE_LENGTH = 5000L; + private static final long BLOCK_SIX_LENGTH = 6000L; + + // data size in bytes + private static final long FILE_ONE_SIZE = 500L; // bin 0 + private static final long FILE_TWO_SIZE = OzoneConsts.KB + 1; // bin 1 + private static final long FILE_THREE_SIZE = 4 * OzoneConsts.KB + 1; // bin 3 + private static final long FILE_FOUR_SIZE = 2 * OzoneConsts.KB + 1; // bin 2 + private static final long FILE_FIVE_SIZE = 100L; // bin 0 + private static final long FILE_EIGHT_SIZE = OzoneConsts.KB + 1; // bin 1 + private static final long FILE_NINE_SIZE = 2 * OzoneConsts.KB + 1; // bin 2 + private static final long FILE_TEN_SIZE = 2 * OzoneConsts.KB + 1; // bin 2 + private static final long FILE_ELEVEN_SIZE = OzoneConsts.KB + 1; // bin 1 + + private static final long FILE1_SIZE_WITH_REPLICA = + getReplicatedSize(FILE_ONE_SIZE, + StandaloneReplicationConfig.getInstance(ONE)); + private static final long FILE2_SIZE_WITH_REPLICA = + getReplicatedSize(FILE_TWO_SIZE, + StandaloneReplicationConfig.getInstance(ONE)); + private static final long FILE3_SIZE_WITH_REPLICA = + getReplicatedSize(FILE_THREE_SIZE, + StandaloneReplicationConfig.getInstance(ONE)); + private static final long FILE4_SIZE_WITH_REPLICA = + getReplicatedSize(FILE_FOUR_SIZE, + StandaloneReplicationConfig.getInstance(ONE)); + private static final long FILE5_SIZE_WITH_REPLICA = + getReplicatedSize(FILE_FIVE_SIZE, + StandaloneReplicationConfig.getInstance(ONE)); + + private static final long FILE8_SIZE_WITH_REPLICA = + getReplicatedSize(FILE_EIGHT_SIZE, + StandaloneReplicationConfig.getInstance(ONE)); + private static final long FILE9_SIZE_WITH_REPLICA = + getReplicatedSize(FILE_NINE_SIZE, + StandaloneReplicationConfig.getInstance(ONE)); + private static final long FILE10_SIZE_WITH_REPLICA = + getReplicatedSize(FILE_TEN_SIZE, + StandaloneReplicationConfig.getInstance(ONE)); + private static final long FILE11_SIZE_WITH_REPLICA = + getReplicatedSize(FILE_ELEVEN_SIZE, + StandaloneReplicationConfig.getInstance(ONE)); + + private static final long MULTI_BLOCK_KEY_SIZE_WITH_REPLICA + = FILE3_SIZE_WITH_REPLICA; + private static final long + MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_ROOT + = FILE1_SIZE_WITH_REPLICA + + FILE2_SIZE_WITH_REPLICA + + FILE3_SIZE_WITH_REPLICA + + FILE4_SIZE_WITH_REPLICA + + FILE5_SIZE_WITH_REPLICA + + FILE8_SIZE_WITH_REPLICA + + FILE9_SIZE_WITH_REPLICA + + FILE10_SIZE_WITH_REPLICA + + FILE11_SIZE_WITH_REPLICA; + + private static final long + MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_VOL + = FILE1_SIZE_WITH_REPLICA + + FILE2_SIZE_WITH_REPLICA + + FILE3_SIZE_WITH_REPLICA + + FILE4_SIZE_WITH_REPLICA + + FILE5_SIZE_WITH_REPLICA; + + private static final long + MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_BUCKET1 + = FILE1_SIZE_WITH_REPLICA + + FILE2_SIZE_WITH_REPLICA + + FILE3_SIZE_WITH_REPLICA; + + private static final long + MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_KEY + = FILE4_SIZE_WITH_REPLICA; + + // quota in bytes + private static final long ROOT_QUOTA = 2 * (2 * OzoneConsts.MB); + private static final long VOL_QUOTA = 2 * OzoneConsts.MB; + private static final long VOL_TWO_QUOTA = 2 * OzoneConsts.MB; + private static final long BUCKET_ONE_QUOTA = OzoneConsts.MB; + private static final long BUCKET_TWO_QUOTA = OzoneConsts.MB; + private static final long BUCKET_THREE_QUOTA = OzoneConsts.MB; + private static final long BUCKET_FOUR_QUOTA = OzoneConsts.MB; + + // mock client's path requests + private static final String TEST_USER = "TestUser"; + private static final String ROOT_PATH = "/"; + private static final String VOL_PATH = ROOT_PATH + VOL; + private static final String VOL_TWO_PATH = ROOT_PATH + VOL_TWO; + private static final String BUCKET_ONE_PATH = + ROOT_PATH + VOL + ROOT_PATH + BUCKET_ONE; + private static final String BUCKET_TWO_PATH = + ROOT_PATH + VOL + ROOT_PATH + BUCKET_TWO; + private static final String KEY_PATH = + ROOT_PATH + VOL + ROOT_PATH + BUCKET_TWO + ROOT_PATH + KEY_FOUR; + private static final String MULTI_BLOCK_KEY_PATH = + ROOT_PATH + VOL + ROOT_PATH + BUCKET_ONE + ROOT_PATH + KEY_THREE; + private static final String INVALID_PATH = "/vol/path/not/found"; + + // some expected answers + private static final long ROOT_DATA_SIZE = + FILE_ONE_SIZE + FILE_TWO_SIZE + FILE_THREE_SIZE + FILE_FOUR_SIZE + + FILE_FIVE_SIZE + FILE_EIGHT_SIZE + FILE_NINE_SIZE + FILE_TEN_SIZE + + FILE_ELEVEN_SIZE; + private static final long VOL_DATA_SIZE = FILE_ONE_SIZE + FILE_TWO_SIZE + + FILE_THREE_SIZE + FILE_FOUR_SIZE + FILE_FIVE_SIZE; + + private static final long VOL_TWO_DATA_SIZE = + FILE_EIGHT_SIZE + FILE_NINE_SIZE + FILE_TEN_SIZE + FILE_ELEVEN_SIZE; + + private static final long BUCKET_ONE_DATA_SIZE = FILE_ONE_SIZE + + FILE_TWO_SIZE + + FILE_THREE_SIZE; + + private static final long BUCKET_TWO_DATA_SIZE = + FILE_FOUR_SIZE + FILE_FIVE_SIZE; + + + @BeforeEach + public void setUp() throws Exception { + conf = new OzoneConfiguration(); + OMMetadataManager omMetadataManager = initializeNewOmMetadataManager( + Files.createDirectory(temporaryFolder.resolve( + "JunitOmDBDir")).toFile(), conf); + OzoneManagerServiceProviderImpl ozoneManagerServiceProvider = + getMockOzoneManagerServiceProvider(); + reconOMMetadataManager = getTestReconOmMetadataManager(omMetadataManager, + Files.createDirectory(temporaryFolder.resolve( + "omMetadatDir")).toFile()); + + ReconTestInjector reconTestInjector = + new ReconTestInjector.Builder(temporaryFolder.toFile()) + .withReconOm(reconOMMetadataManager) + .withOmServiceProvider(ozoneManagerServiceProvider) + .withReconSqlDb() + .withContainerDB() + .addBinding(OzoneStorageContainerManager.class, + getMockReconSCM()) + .addBinding(StorageContainerServiceProvider.class, + mock(StorageContainerServiceProviderImpl.class)) + .addBinding(NSSummaryEndpoint.class) + .build(); + ReconNamespaceSummaryManager reconNamespaceSummaryManager = + reconTestInjector.getInstance(ReconNamespaceSummaryManager.class); + nsSummaryEndpoint = reconTestInjector.getInstance(NSSummaryEndpoint.class); + + // populate OM DB and reprocess into Recon RocksDB + populateOMDB(); + NSSummaryTaskWithOBS nsSummaryTaskWithOBS = + new NSSummaryTaskWithOBS(reconNamespaceSummaryManager, + reconOMMetadataManager, conf); + nsSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); + commonUtils = new CommonUtils(); + } + + @Test + public void testUtility() { + String[] names = EntityHandler.parseRequestPath(TEST_PATH_UTILITY); + assertArrayEquals(TEST_NAMES, names); + String keyName = BucketHandler.getKeyName(names); + assertEquals(TEST_KEY_NAMES, keyName); + String subpath = BucketHandler.buildSubpath(PARENT_DIR, "file1.txt"); + assertEquals(TEST_PATH_UTILITY, subpath); + } + + @Test + public void testGetBasicInfoRoot() throws Exception { + // Test root basics + Response rootResponse = nsSummaryEndpoint.getBasicInfo(ROOT_PATH); + NamespaceSummaryResponse rootResponseObj = + (NamespaceSummaryResponse) rootResponse.getEntity(); + assertEquals(EntityType.ROOT, rootResponseObj.getEntityType()); + assertEquals(2, rootResponseObj.getCountStats().getNumVolume()); + assertEquals(4, rootResponseObj.getCountStats().getNumBucket()); + assertEquals(9, rootResponseObj.getCountStats().getNumTotalKey()); + } + + @Test + public void testGetBasicInfoVol() throws Exception { + // Test volume basics + Response volResponse = nsSummaryEndpoint.getBasicInfo(VOL_PATH); + NamespaceSummaryResponse volResponseObj = + (NamespaceSummaryResponse) volResponse.getEntity(); + assertEquals(EntityType.VOLUME, + volResponseObj.getEntityType()); + assertEquals(2, volResponseObj.getCountStats().getNumBucket()); + assertEquals(5, volResponseObj.getCountStats().getNumTotalKey()); + assertEquals(TEST_USER, ((VolumeObjectDBInfo) volResponseObj. + getObjectDBInfo()).getAdmin()); + assertEquals(TEST_USER, ((VolumeObjectDBInfo) volResponseObj. + getObjectDBInfo()).getOwner()); + assertEquals(VOL, volResponseObj.getObjectDBInfo().getName()); + assertEquals(2097152, volResponseObj.getObjectDBInfo().getQuotaInBytes()); + assertEquals(-1, volResponseObj.getObjectDBInfo().getQuotaInNamespace()); + } + + @Test + public void testGetBasicInfoBucketOne() throws Exception { + // Test bucket 1's basics + Response bucketOneResponse = + nsSummaryEndpoint.getBasicInfo(BUCKET_ONE_PATH); + NamespaceSummaryResponse bucketOneObj = + (NamespaceSummaryResponse) bucketOneResponse.getEntity(); + assertEquals(EntityType.BUCKET, bucketOneObj.getEntityType()); + assertEquals(3, bucketOneObj.getCountStats().getNumTotalKey()); + assertEquals(VOL, + ((BucketObjectDBInfo) bucketOneObj.getObjectDBInfo()).getVolumeName()); + assertEquals(StorageType.DISK, + ((BucketObjectDBInfo) + bucketOneObj.getObjectDBInfo()).getStorageType()); + assertEquals(getBucketLayout(), + ((BucketObjectDBInfo) + bucketOneObj.getObjectDBInfo()).getBucketLayout()); + assertEquals(BUCKET_ONE, + ((BucketObjectDBInfo) bucketOneObj.getObjectDBInfo()).getName()); + } + + @Test + public void testGetBasicInfoBucketTwo() throws Exception { + // Test bucket 2's basics + commonUtils.testNSSummaryBasicInfoBucketTwo( + BucketLayout.OBJECT_STORE, + nsSummaryEndpoint); + } + + @Test + public void testGetBasicInfoNoPath() throws Exception { + // Test invalid path + commonUtils.testNSSummaryBasicInfoNoPath(nsSummaryEndpoint); + } + + @Test + public void testGetBasicInfoKey() throws Exception { + // Test key + commonUtils.testNSSummaryBasicInfoKey(nsSummaryEndpoint); + } + + @Test + public void testDiskUsageRoot() throws Exception { + // root level DU + Response rootResponse = nsSummaryEndpoint.getDiskUsage(ROOT_PATH, + false, false); + DUResponse duRootRes = (DUResponse) rootResponse.getEntity(); + assertEquals(2, duRootRes.getCount()); + List duRootData = duRootRes.getDuData(); + // sort based on subpath + Collections.sort(duRootData, + Comparator.comparing(DUResponse.DiskUsage::getSubpath)); + DUResponse.DiskUsage duVol1 = duRootData.get(0); + DUResponse.DiskUsage duVol2 = duRootData.get(1); + assertEquals(VOL_PATH, duVol1.getSubpath()); + assertEquals(VOL_TWO_PATH, duVol2.getSubpath()); + assertEquals(VOL_DATA_SIZE, duVol1.getSize()); + assertEquals(VOL_TWO_DATA_SIZE, duVol2.getSize()); + } + + @Test + public void testDiskUsageVolume() throws Exception { + // volume level DU + Response volResponse = nsSummaryEndpoint.getDiskUsage(VOL_PATH, + false, false); + DUResponse duVolRes = (DUResponse) volResponse.getEntity(); + assertEquals(2, duVolRes.getCount()); + List duData = duVolRes.getDuData(); + // sort based on subpath + Collections.sort(duData, + Comparator.comparing(DUResponse.DiskUsage::getSubpath)); + DUResponse.DiskUsage duBucket1 = duData.get(0); + DUResponse.DiskUsage duBucket2 = duData.get(1); + assertEquals(BUCKET_ONE_PATH, duBucket1.getSubpath()); + assertEquals(BUCKET_TWO_PATH, duBucket2.getSubpath()); + assertEquals(BUCKET_ONE_DATA_SIZE, duBucket1.getSize()); + assertEquals(BUCKET_TWO_DATA_SIZE, duBucket2.getSize()); + } + + @Test + public void testDiskUsageBucket() throws Exception { + // bucket level DU + Response bucketResponse = nsSummaryEndpoint.getDiskUsage(BUCKET_ONE_PATH, + false, false); + DUResponse duBucketResponse = (DUResponse) bucketResponse.getEntity(); + // There are no sub-paths under this OBS bucket. + assertEquals(0, duBucketResponse.getCount()); + assertEquals(BUCKET_ONE_DATA_SIZE, duBucketResponse.getSize()); + } + + @Test + public void testDiskUsageKey() throws Exception { + // key level DU + Response keyResponse = nsSummaryEndpoint.getDiskUsage(KEY_PATH, + false, false); + DUResponse keyObj = (DUResponse) keyResponse.getEntity(); + assertEquals(0, keyObj.getCount()); + assertEquals(FILE_FOUR_SIZE, keyObj.getSize()); + } + + @Test + public void testDiskUsageUnknown() throws Exception { + // invalid path check + Response invalidResponse = nsSummaryEndpoint.getDiskUsage(INVALID_PATH, + false, false); + DUResponse invalidObj = (DUResponse) invalidResponse.getEntity(); + assertEquals(ResponseStatus.PATH_NOT_FOUND, + invalidObj.getStatus()); + } + + @Test + public void testDiskUsageWithReplication() throws Exception { + setUpMultiBlockKey(); + Response keyResponse = nsSummaryEndpoint.getDiskUsage(MULTI_BLOCK_KEY_PATH, + false, true); + DUResponse replicaDUResponse = (DUResponse) keyResponse.getEntity(); + assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus()); + assertEquals(MULTI_BLOCK_KEY_SIZE_WITH_REPLICA, + replicaDUResponse.getSizeWithReplica()); + } + + @Test + public void testDataSizeUnderRootWithReplication() throws IOException { + setUpMultiBlockReplicatedKeys(); + // withReplica is true + Response rootResponse = nsSummaryEndpoint.getDiskUsage(ROOT_PATH, + false, true); + DUResponse replicaDUResponse = (DUResponse) rootResponse.getEntity(); + assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus()); + assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_ROOT, + replicaDUResponse.getSizeWithReplica()); + assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_VOL, + replicaDUResponse.getDuData().get(0).getSizeWithReplica()); + + } + + @Test + public void testDataSizeUnderVolWithReplication() throws IOException { + setUpMultiBlockReplicatedKeys(); + Response volResponse = nsSummaryEndpoint.getDiskUsage(VOL_PATH, + false, true); + DUResponse replicaDUResponse = (DUResponse) volResponse.getEntity(); + assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus()); + assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_VOL, + replicaDUResponse.getSizeWithReplica()); + assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_BUCKET1, + replicaDUResponse.getDuData().get(0).getSizeWithReplica()); + } + + @Test + public void testDataSizeUnderBucketWithReplication() throws IOException { + setUpMultiBlockReplicatedKeys(); + Response bucketResponse = nsSummaryEndpoint.getDiskUsage(BUCKET_ONE_PATH, + false, true); + DUResponse replicaDUResponse = (DUResponse) bucketResponse.getEntity(); + assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus()); + assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_BUCKET1, + replicaDUResponse.getSizeWithReplica()); + } + + @Test + public void testDataSizeUnderKeyWithReplication() throws IOException { + setUpMultiBlockReplicatedKeys(); + Response keyResponse = nsSummaryEndpoint.getDiskUsage(KEY_PATH, + false, true); + DUResponse replicaDUResponse = (DUResponse) keyResponse.getEntity(); + assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus()); + assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_KEY, + replicaDUResponse.getSizeWithReplica()); + } + + @Test + public void testQuotaUsage() throws Exception { + // root level quota usage + Response rootResponse = nsSummaryEndpoint.getQuotaUsage(ROOT_PATH); + QuotaUsageResponse quRootRes = + (QuotaUsageResponse) rootResponse.getEntity(); + assertEquals(ROOT_QUOTA, quRootRes.getQuota()); + assertEquals(ROOT_DATA_SIZE, quRootRes.getQuotaUsed()); + + // volume level quota usage + Response volResponse = nsSummaryEndpoint.getQuotaUsage(VOL_PATH); + QuotaUsageResponse quVolRes = (QuotaUsageResponse) volResponse.getEntity(); + assertEquals(VOL_QUOTA, quVolRes.getQuota()); + assertEquals(VOL_DATA_SIZE, quVolRes.getQuotaUsed()); + + // bucket level quota usage + Response bucketRes = nsSummaryEndpoint.getQuotaUsage(BUCKET_ONE_PATH); + QuotaUsageResponse quBucketRes = (QuotaUsageResponse) bucketRes.getEntity(); + assertEquals(BUCKET_ONE_QUOTA, quBucketRes.getQuota()); + assertEquals(BUCKET_ONE_DATA_SIZE, quBucketRes.getQuotaUsed()); + + Response bucketRes2 = nsSummaryEndpoint.getQuotaUsage(BUCKET_TWO_PATH); + QuotaUsageResponse quBucketRes2 = + (QuotaUsageResponse) bucketRes2.getEntity(); + assertEquals(BUCKET_TWO_QUOTA, quBucketRes2.getQuota()); + assertEquals(BUCKET_TWO_DATA_SIZE, quBucketRes2.getQuotaUsed()); + + // other level not applicable + Response naResponse2 = nsSummaryEndpoint.getQuotaUsage(KEY_PATH); + QuotaUsageResponse quotaUsageResponse2 = + (QuotaUsageResponse) naResponse2.getEntity(); + assertEquals(ResponseStatus.TYPE_NOT_APPLICABLE, + quotaUsageResponse2.getResponseCode()); + + // invalid path request + Response invalidRes = nsSummaryEndpoint.getQuotaUsage(INVALID_PATH); + QuotaUsageResponse invalidResObj = + (QuotaUsageResponse) invalidRes.getEntity(); + assertEquals(ResponseStatus.PATH_NOT_FOUND, + invalidResObj.getResponseCode()); + } + + + @Test + public void testFileSizeDist() throws Exception { + checkFileSizeDist(ROOT_PATH, 2, 3, 3, 1); + checkFileSizeDist(VOL_PATH, 2, 1, 1, 1); + checkFileSizeDist(BUCKET_ONE_PATH, 1, 1, 0, 1); + } + + public void checkFileSizeDist(String path, int bin0, + int bin1, int bin2, int bin3) throws Exception { + Response res = nsSummaryEndpoint.getFileSizeDistribution(path); + FileSizeDistributionResponse fileSizeDistResObj = + (FileSizeDistributionResponse) res.getEntity(); + int[] fileSizeDist = fileSizeDistResObj.getFileSizeDist(); + assertEquals(bin0, fileSizeDist[0]); + assertEquals(bin1, fileSizeDist[1]); + assertEquals(bin2, fileSizeDist[2]); + assertEquals(bin3, fileSizeDist[3]); + for (int i = 4; i < ReconConstants.NUM_OF_FILE_SIZE_BINS; ++i) { + assertEquals(0, fileSizeDist[i]); + } + } + + /** + * Testing the following case. + * ├── vol + * │ ├── bucket1 + * │ │ ├── file1 + * │ │ └── file2 + * │ │ └── file3 + * │ └── bucket2 + * │ ├── file4 + * │ └── file5 + * └── vol2 + * ├── bucket3 + * │ ├── file8 + * │ ├── file9 + * │ └── file10 + * └── bucket4 + * └── file11 + * + * Write these keys to OM and + * replicate them. + */ + @SuppressWarnings("checkstyle:MethodLength") + private void populateOMDB() throws Exception { + + // write all keys + writeKeyToOm(reconOMMetadataManager, + KEY_ONE, + BUCKET_ONE, + VOL, + KEY_ONE, + KEY_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + VOL_OBJECT_ID, + FILE_ONE_SIZE, + getBucketLayout()); + writeKeyToOm(reconOMMetadataManager, + KEY_TWO, + BUCKET_ONE, + VOL, + KEY_TWO, + KEY_TWO_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + VOL_OBJECT_ID, + FILE_TWO_SIZE, + getBucketLayout()); + writeKeyToOm(reconOMMetadataManager, + KEY_THREE, + BUCKET_ONE, + VOL, + KEY_THREE, + KEY_THREE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + VOL_OBJECT_ID, + FILE_THREE_SIZE, + getBucketLayout()); + writeKeyToOm(reconOMMetadataManager, + KEY_FOUR, + BUCKET_TWO, + VOL, + KEY_FOUR, + KEY_FOUR_OBJECT_ID, + BUCKET_TWO_OBJECT_ID, + BUCKET_TWO_OBJECT_ID, + VOL_OBJECT_ID, + FILE_FOUR_SIZE, + getBucketLayout()); + writeKeyToOm(reconOMMetadataManager, + KEY_FIVE, + BUCKET_TWO, + VOL, + KEY_FIVE, + KEY_FIVE_OBJECT_ID, + BUCKET_TWO_OBJECT_ID, + BUCKET_TWO_OBJECT_ID, + VOL_OBJECT_ID, + FILE_FIVE_SIZE, + getBucketLayout()); + + writeKeyToOm(reconOMMetadataManager, + KEY_EIGHT, + BUCKET_THREE, + VOL_TWO, + KEY_EIGHT, + KEY_EIGHT_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + VOL_TWO_OBJECT_ID, + FILE_EIGHT_SIZE, + getBucketLayout()); + writeKeyToOm(reconOMMetadataManager, + KEY_NINE, + BUCKET_THREE, + VOL_TWO, + KEY_NINE, + KEY_NINE_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + VOL_TWO_OBJECT_ID, + FILE_NINE_SIZE, + getBucketLayout()); + writeKeyToOm(reconOMMetadataManager, + KEY_TEN, + BUCKET_THREE, + VOL_TWO, + KEY_TEN, + KEY_TEN_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + VOL_TWO_OBJECT_ID, + FILE_TEN_SIZE, + getBucketLayout()); + writeKeyToOm(reconOMMetadataManager, + KEY_ELEVEN, + BUCKET_FOUR, + VOL_TWO, + KEY_ELEVEN, + KEY_ELEVEN_OBJECT_ID, + PARENT_OBJECT_ID_ZERO, + BUCKET_FOUR_OBJECT_ID, + VOL_TWO_OBJECT_ID, + FILE_ELEVEN_SIZE, + getBucketLayout()); + } + + /** + * Create a new OM Metadata manager instance with one user, one vol, and two + * buckets. + * + * @throws IOException ioEx + */ + private static OMMetadataManager initializeNewOmMetadataManager( + File omDbDir, OzoneConfiguration omConfiguration) + throws IOException { + omConfiguration.set(OZONE_OM_DB_DIRS, + omDbDir.getAbsolutePath()); + omConfiguration.set(OMConfigKeys + .OZONE_OM_ENABLE_FILESYSTEM_PATHS, "true"); + OMMetadataManager omMetadataManager = new OmMetadataManagerImpl( + omConfiguration, null); + + String volumeKey = omMetadataManager.getVolumeKey(VOL); + OmVolumeArgs args = + OmVolumeArgs.newBuilder() + .setObjectID(VOL_OBJECT_ID) + .setVolume(VOL) + .setAdminName(TEST_USER) + .setOwnerName(TEST_USER) + .setQuotaInBytes(VOL_QUOTA) + .build(); + + String volume2Key = omMetadataManager.getVolumeKey(VOL_TWO); + OmVolumeArgs args2 = + OmVolumeArgs.newBuilder() + .setObjectID(VOL_TWO_OBJECT_ID) + .setVolume(VOL_TWO) + .setAdminName(TEST_USER) + .setOwnerName(TEST_USER) + .setQuotaInBytes(VOL_TWO_QUOTA) + .build(); + + omMetadataManager.getVolumeTable().put(volumeKey, args); + omMetadataManager.getVolumeTable().put(volume2Key, args2); + + OmBucketInfo bucketInfo = OmBucketInfo.newBuilder() + .setVolumeName(VOL) + .setBucketName(BUCKET_ONE) + .setObjectID(BUCKET_ONE_OBJECT_ID) + .setQuotaInBytes(BUCKET_ONE_QUOTA) + .setBucketLayout(getBucketLayout()) + .build(); + + OmBucketInfo bucketInfo2 = OmBucketInfo.newBuilder() + .setVolumeName(VOL) + .setBucketName(BUCKET_TWO) + .setObjectID(BUCKET_TWO_OBJECT_ID) + .setQuotaInBytes(BUCKET_TWO_QUOTA) + .setBucketLayout(getBucketLayout()) + .build(); + + OmBucketInfo bucketInfo3 = OmBucketInfo.newBuilder() + .setVolumeName(VOL_TWO) + .setBucketName(BUCKET_THREE) + .setObjectID(BUCKET_THREE_OBJECT_ID) + .setQuotaInBytes(BUCKET_THREE_QUOTA) + .setBucketLayout(getBucketLayout()) + .build(); + + OmBucketInfo bucketInfo4 = OmBucketInfo.newBuilder() + .setVolumeName(VOL_TWO) + .setBucketName(BUCKET_FOUR) + .setObjectID(BUCKET_FOUR_OBJECT_ID) + .setQuotaInBytes(BUCKET_FOUR_QUOTA) + .setBucketLayout(getBucketLayout()) + .build(); + + String bucketKey = omMetadataManager.getBucketKey( + bucketInfo.getVolumeName(), bucketInfo.getBucketName()); + String bucketKey2 = omMetadataManager.getBucketKey( + bucketInfo2.getVolumeName(), bucketInfo2.getBucketName()); + String bucketKey3 = omMetadataManager.getBucketKey( + bucketInfo3.getVolumeName(), bucketInfo3.getBucketName()); + String bucketKey4 = omMetadataManager.getBucketKey( + bucketInfo4.getVolumeName(), bucketInfo4.getBucketName()); + + omMetadataManager.getBucketTable().put(bucketKey, bucketInfo); + omMetadataManager.getBucketTable().put(bucketKey2, bucketInfo2); + omMetadataManager.getBucketTable().put(bucketKey3, bucketInfo3); + omMetadataManager.getBucketTable().put(bucketKey4, bucketInfo4); + + return omMetadataManager; + } + + private void setUpMultiBlockKey() throws IOException { + OmKeyLocationInfoGroup locationInfoGroup = + getLocationInfoGroup1(); + + // add the multi-block key to Recon's OM + writeKeyToOm(reconOMMetadataManager, + MULTI_BLOCK_FILE, + BUCKET_ONE, + VOL, + MULTI_BLOCK_FILE, + MULTI_BLOCK_KEY_OBJECT_ID, + PARENT_OBJECT_ID_ZERO, + BUCKET_ONE_OBJECT_ID, + VOL_OBJECT_ID, + Collections.singletonList(locationInfoGroup), + getBucketLayout(), + FILE_THREE_SIZE); + } + + private OmKeyLocationInfoGroup getLocationInfoGroup1() { + List locationInfoList = new ArrayList<>(); + BlockID block1 = new BlockID(CONTAINER_ONE_ID, 0L); + BlockID block2 = new BlockID(CONTAINER_TWO_ID, 0L); + BlockID block3 = new BlockID(CONTAINER_THREE_ID, 0L); + + OmKeyLocationInfo location1 = new OmKeyLocationInfo.Builder() + .setBlockID(block1) + .setLength(BLOCK_ONE_LENGTH) + .build(); + OmKeyLocationInfo location2 = new OmKeyLocationInfo.Builder() + .setBlockID(block2) + .setLength(BLOCK_TWO_LENGTH) + .build(); + OmKeyLocationInfo location3 = new OmKeyLocationInfo.Builder() + .setBlockID(block3) + .setLength(BLOCK_THREE_LENGTH) + .build(); + locationInfoList.add(location1); + locationInfoList.add(location2); + locationInfoList.add(location3); + + return new OmKeyLocationInfoGroup(0L, locationInfoList); + } + + + private OmKeyLocationInfoGroup getLocationInfoGroup2() { + List locationInfoList = new ArrayList<>(); + BlockID block4 = new BlockID(CONTAINER_FOUR_ID, 0L); + BlockID block5 = new BlockID(CONTAINER_FIVE_ID, 0L); + BlockID block6 = new BlockID(CONTAINER_SIX_ID, 0L); + + OmKeyLocationInfo location4 = new OmKeyLocationInfo.Builder() + .setBlockID(block4) + .setLength(BLOCK_FOUR_LENGTH) + .build(); + OmKeyLocationInfo location5 = new OmKeyLocationInfo.Builder() + .setBlockID(block5) + .setLength(BLOCK_FIVE_LENGTH) + .build(); + OmKeyLocationInfo location6 = new OmKeyLocationInfo.Builder() + .setBlockID(block6) + .setLength(BLOCK_SIX_LENGTH) + .build(); + locationInfoList.add(location4); + locationInfoList.add(location5); + locationInfoList.add(location6); + return new OmKeyLocationInfoGroup(0L, locationInfoList); + + } + + @SuppressWarnings("checkstyle:MethodLength") + private void setUpMultiBlockReplicatedKeys() throws IOException { + OmKeyLocationInfoGroup locationInfoGroup1 = + getLocationInfoGroup1(); + OmKeyLocationInfoGroup locationInfoGroup2 = + getLocationInfoGroup2(); + + //vol/bucket1/file1 + writeKeyToOm(reconOMMetadataManager, + KEY_ONE, + BUCKET_ONE, + VOL, + KEY_ONE, + KEY_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + VOL_OBJECT_ID, + Collections.singletonList(locationInfoGroup1), + getBucketLayout(), + FILE_ONE_SIZE); + + //vol/bucket1/file2 + writeKeyToOm(reconOMMetadataManager, + KEY_TWO, + BUCKET_ONE, + VOL, + KEY_TWO, + KEY_TWO_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + VOL_OBJECT_ID, + Collections.singletonList(locationInfoGroup2), + getBucketLayout(), + FILE_TWO_SIZE); + + //vol/bucket1/file3 + writeKeyToOm(reconOMMetadataManager, + KEY_THREE, + BUCKET_ONE, + VOL, + KEY_THREE, + KEY_THREE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + VOL_OBJECT_ID, + Collections.singletonList(locationInfoGroup1), + getBucketLayout(), + FILE_THREE_SIZE); + + //vol/bucket2/file4 + writeKeyToOm(reconOMMetadataManager, + KEY_FOUR, + BUCKET_TWO, + VOL, + KEY_FOUR, + KEY_FOUR_OBJECT_ID, + BUCKET_TWO_OBJECT_ID, + BUCKET_TWO_OBJECT_ID, + VOL_OBJECT_ID, + Collections.singletonList(locationInfoGroup2), + getBucketLayout(), + FILE_FOUR_SIZE); + + //vol/bucket2/file5 + writeKeyToOm(reconOMMetadataManager, + KEY_FIVE, + BUCKET_TWO, + VOL, + KEY_FIVE, + KEY_FIVE_OBJECT_ID, + BUCKET_TWO_OBJECT_ID, + BUCKET_TWO_OBJECT_ID, + VOL_OBJECT_ID, + Collections.singletonList(locationInfoGroup1), + getBucketLayout(), + FILE_FIVE_SIZE); + + //vol2/bucket3/file8 + writeKeyToOm(reconOMMetadataManager, + KEY_EIGHT, + BUCKET_THREE, + VOL_TWO, + KEY_EIGHT, + KEY_EIGHT_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + VOL_TWO_OBJECT_ID, + Collections.singletonList(locationInfoGroup2), + getBucketLayout(), + FILE_EIGHT_SIZE); + + //vol2/bucket3/file9 + writeKeyToOm(reconOMMetadataManager, + KEY_NINE, + BUCKET_THREE, + VOL_TWO, + KEY_NINE, + KEY_NINE_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + VOL_TWO_OBJECT_ID, + Collections.singletonList(locationInfoGroup1), + getBucketLayout(), + FILE_NINE_SIZE); + + //vol2/bucket3/file10 + writeKeyToOm(reconOMMetadataManager, + KEY_TEN, + BUCKET_THREE, + VOL_TWO, + KEY_TEN, + KEY_TEN_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + BUCKET_THREE_OBJECT_ID, + VOL_TWO_OBJECT_ID, + Collections.singletonList(locationInfoGroup2), + getBucketLayout(), + FILE_TEN_SIZE); + + //vol2/bucket4/file11 + writeKeyToOm(reconOMMetadataManager, + KEY_ELEVEN, + BUCKET_FOUR, + VOL_TWO, + KEY_ELEVEN, + KEY_ELEVEN_OBJECT_ID, + BUCKET_FOUR_OBJECT_ID, + BUCKET_FOUR_OBJECT_ID, + VOL_TWO_OBJECT_ID, + Collections.singletonList(locationInfoGroup1), + getBucketLayout(), + FILE_ELEVEN_SIZE); + } + + /** + * Generate a set of mock container replica with a size of + * replication factor for container. + * + * @param replicationFactor number of replica + * @param containerID the container replicated based upon + * @return a set of container replica for testing + */ + private static Set generateMockContainerReplicas( + int replicationFactor, ContainerID containerID) { + Set result = new HashSet<>(); + for (int i = 0; i < replicationFactor; ++i) { + DatanodeDetails randomDatanode = randomDatanodeDetails(); + ContainerReplica replica = new ContainerReplica.ContainerReplicaBuilder() + .setContainerID(containerID) + .setContainerState( + StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State.OPEN) + .setDatanodeDetails(randomDatanode) + .build(); + result.add(replica); + } + return result; + } + + private static ReconStorageContainerManagerFacade getMockReconSCM() + throws ContainerNotFoundException { + ReconStorageContainerManagerFacade reconSCM = + mock(ReconStorageContainerManagerFacade.class); + ContainerManager containerManager = mock(ContainerManager.class); + + // Container 1 is 3-way replicated + ContainerID containerID1 = new ContainerID(CONTAINER_ONE_ID); + Set containerReplicas1 = generateMockContainerReplicas( + CONTAINER_ONE_REPLICA_COUNT, containerID1); + when(containerManager.getContainerReplicas(containerID1)) + .thenReturn(containerReplicas1); + + // Container 2 is under replicated with 2 replica + ContainerID containerID2 = new ContainerID(CONTAINER_TWO_ID); + Set containerReplicas2 = generateMockContainerReplicas( + CONTAINER_TWO_REPLICA_COUNT, containerID2); + when(containerManager.getContainerReplicas(containerID2)) + .thenReturn(containerReplicas2); + + // Container 3 is over replicated with 4 replica + ContainerID containerID3 = new ContainerID(CONTAINER_THREE_ID); + Set containerReplicas3 = generateMockContainerReplicas( + CONTAINER_THREE_REPLICA_COUNT, containerID3); + when(containerManager.getContainerReplicas(containerID3)) + .thenReturn(containerReplicas3); + + // Container 4 is replicated with 5 replica + ContainerID containerID4 = new ContainerID(CONTAINER_FOUR_ID); + Set containerReplicas4 = generateMockContainerReplicas( + CONTAINER_FOUR_REPLICA_COUNT, containerID4); + when(containerManager.getContainerReplicas(containerID4)) + .thenReturn(containerReplicas4); + + // Container 5 is replicated with 2 replica + ContainerID containerID5 = new ContainerID(CONTAINER_FIVE_ID); + Set containerReplicas5 = generateMockContainerReplicas( + CONTAINER_FIVE_REPLICA_COUNT, containerID5); + when(containerManager.getContainerReplicas(containerID5)) + .thenReturn(containerReplicas5); + + // Container 6 is replicated with 3 replica + ContainerID containerID6 = new ContainerID(CONTAINER_SIX_ID); + Set containerReplicas6 = generateMockContainerReplicas( + CONTAINER_SIX_REPLICA_COUNT, containerID6); + when(containerManager.getContainerReplicas(containerID6)) + .thenReturn(containerReplicas6); + + when(reconSCM.getContainerManager()).thenReturn(containerManager); + ReconNodeManager mockReconNodeManager = mock(ReconNodeManager.class); + when(mockReconNodeManager.getStats()).thenReturn(getMockSCMRootStat()); + when(reconSCM.getScmNodeManager()).thenReturn(mockReconNodeManager); + return reconSCM; + } + + private static BucketLayout getBucketLayout() { + return BucketLayout.OBJECT_STORE; + } + + private static SCMNodeStat getMockSCMRootStat() { + return new SCMNodeStat(ROOT_QUOTA, ROOT_DATA_SIZE, + ROOT_QUOTA - ROOT_DATA_SIZE, 0L, 0L); + } + +} diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java index 6992c3100fb9..485804240d52 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java @@ -166,7 +166,7 @@ public void setUp() throws Exception { reconNamespaceSummaryManager.getNSSummary(BUCKET_THREE_OBJECT_ID); assertNotNull(nsSummaryForBucket1); assertNotNull(nsSummaryForBucket2); - assertNull(nsSummaryForBucket3); + assertNotNull(nsSummaryForBucket3); } @Test @@ -233,7 +233,7 @@ public void setUp() throws IOException { assertNotNull(nsSummaryForBucket2); nsSummaryForBucket3 = reconNamespaceSummaryManager.getNSSummary(BUCKET_THREE_OBJECT_ID); - assertNull(nsSummaryForBucket3); + assertNotNull(nsSummaryForBucket3); } private OMUpdateEventBatch processEventBatch() throws IOException { diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithOBS.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithOBS.java new file mode 100644 index 000000000000..8f9d6b2990a5 --- /dev/null +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithOBS.java @@ -0,0 +1,548 @@ +/** + * 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.recon.tasks; + +import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; +import org.apache.hadoop.ozone.om.OMConfigKeys; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.recon.ReconConstants; +import org.apache.hadoop.ozone.recon.ReconTestInjector; +import org.apache.hadoop.ozone.recon.api.types.NSSummary; +import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; +import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; +import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Set; + +import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_DB_DIRS; +import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager; +import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm; +import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProviderWithFSO; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; + +/** + * Unit test for NSSummaryTaskWithOBS. + */ +public final class TestNSSummaryTaskWithOBS implements Serializable { + private static ReconNamespaceSummaryManager reconNamespaceSummaryManager; + private static OMMetadataManager omMetadataManager; + private static ReconOMMetadataManager reconOMMetadataManager; + private static NSSummaryTaskWithOBS nSSummaryTaskWithOBS; + private static OzoneConfiguration omConfiguration; + + // Object names + private static final String VOL = "vol"; + private static final String BUCKET_ONE = "bucket1"; + private static final String BUCKET_TWO = "bucket2"; + private static final String KEY_ONE = "key1"; + private static final String KEY_TWO = "key2"; + private static final String KEY_THREE = "dir1/dir2/key3"; + private static final String KEY_FOUR = "key4///////////"; + private static final String KEY_FIVE = "//////////"; + private static final String KEY_SIX = "key6"; + private static final String KEY_SEVEN = "key7"; + + private static final String TEST_USER = "TestUser"; + + private static final long PARENT_OBJECT_ID_ZERO = 0L; + private static final long VOL_OBJECT_ID = 0L; + private static final long BUCKET_ONE_OBJECT_ID = 1L; + private static final long BUCKET_TWO_OBJECT_ID = 2L; + private static final long KEY_ONE_OBJECT_ID = 3L; + private static final long KEY_TWO_OBJECT_ID = 5L; + private static final long KEY_FOUR_OBJECT_ID = 6L; + private static final long KEY_THREE_OBJECT_ID = 8L; + private static final long KEY_FIVE_OBJECT_ID = 9L; + private static final long KEY_SIX_OBJECT_ID = 10L; + private static final long KEY_SEVEN_OBJECT_ID = 11L; + + + private static final long KEY_ONE_SIZE = 500L; + private static final long KEY_TWO_OLD_SIZE = 1025L; + private static final long KEY_TWO_UPDATE_SIZE = 1023L; + private static final long KEY_THREE_SIZE = + ReconConstants.MAX_FILE_SIZE_UPPER_BOUND - 100L; + private static final long KEY_FOUR_SIZE = 2050L; + private static final long KEY_FIVE_SIZE = 100L; + private static final long KEY_SIX_SIZE = 6000L; + private static final long KEY_SEVEN_SIZE = 7000L; + + private TestNSSummaryTaskWithOBS() { + } + + @BeforeAll + public static void setUp(@TempDir File tmpDir) throws Exception { + initializeNewOmMetadataManager(new File(tmpDir, "om")); + OzoneManagerServiceProviderImpl ozoneManagerServiceProvider = + getMockOzoneManagerServiceProviderWithFSO(); + reconOMMetadataManager = getTestReconOmMetadataManager(omMetadataManager, + new File(tmpDir, "recon")); + + ReconTestInjector reconTestInjector = + new ReconTestInjector.Builder(tmpDir) + .withReconOm(reconOMMetadataManager) + .withOmServiceProvider(ozoneManagerServiceProvider) + .withReconSqlDb() + .withContainerDB() + .build(); + reconNamespaceSummaryManager = + reconTestInjector.getInstance(ReconNamespaceSummaryManager.class); + + NSSummary nonExistentSummary = + reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID); + assertNull(nonExistentSummary); + + populateOMDB(); + + nSSummaryTaskWithOBS = new NSSummaryTaskWithOBS( + reconNamespaceSummaryManager, + reconOMMetadataManager, omConfiguration); + } + + /** + * Nested class for testing NSSummaryTaskWithOBS reprocess. + */ + @Nested + public class TestReprocess { + + private NSSummary nsSummaryForBucket1; + private NSSummary nsSummaryForBucket2; + + @BeforeEach + public void setUp() throws IOException { + // write a NSSummary prior to reprocess + // verify it got cleaned up after. + NSSummary staleNSSummary = new NSSummary(); + RDBBatchOperation rdbBatchOperation = new RDBBatchOperation(); + reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation, -1L, + staleNSSummary); + reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation); + + // Verify commit + assertNotNull(reconNamespaceSummaryManager.getNSSummary(-1L)); + + // reinit Recon RocksDB's namespace CF. + reconNamespaceSummaryManager.clearNSSummaryTable(); + + nSSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); + assertNull(reconNamespaceSummaryManager.getNSSummary(-1L)); + + nsSummaryForBucket1 = + reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID); + nsSummaryForBucket2 = + reconNamespaceSummaryManager.getNSSummary(BUCKET_TWO_OBJECT_ID); + assertNotNull(nsSummaryForBucket1); + assertNotNull(nsSummaryForBucket2); + } + + @Test + public void testReprocessNSSummaryNull() throws IOException { + assertNull(reconNamespaceSummaryManager.getNSSummary(-1L)); + } + + @Test + public void testReprocessGetFiles() { + assertEquals(3, nsSummaryForBucket1.getNumOfFiles()); + assertEquals(2, nsSummaryForBucket2.getNumOfFiles()); + + assertEquals(KEY_ONE_SIZE + KEY_TWO_OLD_SIZE + KEY_THREE_SIZE, + nsSummaryForBucket1.getSizeOfFiles()); + assertEquals(KEY_FOUR_SIZE + KEY_FIVE_SIZE, + nsSummaryForBucket2.getSizeOfFiles()); + } + + @Test + public void testReprocessFileBucketSize() { + int[] fileDistBucket1 = nsSummaryForBucket1.getFileSizeBucket(); + int[] fileDistBucket2 = nsSummaryForBucket2.getFileSizeBucket(); + assertEquals(ReconConstants.NUM_OF_FILE_SIZE_BINS, + fileDistBucket1.length); + assertEquals(ReconConstants.NUM_OF_FILE_SIZE_BINS, + fileDistBucket2.length); + + // Check for 1's and 0's in fileDistBucket1 + int[] expectedIndexes1 = {0, 1, 40}; + for (int index = 0; index < fileDistBucket1.length; index++) { + if (contains(expectedIndexes1, index)) { + assertEquals(1, fileDistBucket1[index]); + } else { + assertEquals(0, fileDistBucket1[index]); + } + } + + // Check for 1's and 0's in fileDistBucket2 + int[] expectedIndexes2 = {0, 2}; + for (int index = 0; index < fileDistBucket2.length; index++) { + if (contains(expectedIndexes2, index)) { + assertEquals(1, fileDistBucket2[index]); + } else { + assertEquals(0, fileDistBucket2[index]); + } + } + } + + } + + /** + * Nested class for testing NSSummaryTaskWithOBS process. + */ + @Nested + public class TestProcess { + + private NSSummary nsSummaryForBucket1; + private NSSummary nsSummaryForBucket2; + + private OMDBUpdateEvent keyEvent1; + private OMDBUpdateEvent keyEvent2; + private OMDBUpdateEvent keyEvent3; + private OMDBUpdateEvent keyEvent4; + + @BeforeEach + public void setUp() throws IOException { + // reinit Recon RocksDB's namespace CF. + reconNamespaceSummaryManager.clearNSSummaryTable(); + nSSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); + nSSummaryTaskWithOBS.processWithOBS(processEventBatch()); + + nsSummaryForBucket1 = + reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID); + assertNotNull(nsSummaryForBucket1); + nsSummaryForBucket2 = + reconNamespaceSummaryManager.getNSSummary(BUCKET_TWO_OBJECT_ID); + assertNotNull(nsSummaryForBucket2); + } + + private OMUpdateEventBatch processEventBatch() throws IOException { + // Test PUT Event. + // PUT Key6 in Bucket2. + String omPutKey = + OM_KEY_PREFIX + VOL + + OM_KEY_PREFIX + BUCKET_TWO + + OM_KEY_PREFIX + KEY_SIX; + OmKeyInfo omPutKeyInfo = buildOmKeyInfo(VOL, BUCKET_TWO, KEY_SIX, + KEY_SIX, KEY_SIX_OBJECT_ID, BUCKET_TWO_OBJECT_ID, KEY_SIX_SIZE); + keyEvent1 = new OMDBUpdateEvent. + OMUpdateEventBuilder() + .setKey(omPutKey) + .setValue(omPutKeyInfo) + .setTable(omMetadataManager.getKeyTable(getBucketLayout()) + .getName()) + .setAction(OMDBUpdateEvent.OMDBUpdateAction.PUT) + .build(); + // PUT Key7 in Bucket1. + omPutKey = + OM_KEY_PREFIX + VOL + + OM_KEY_PREFIX + BUCKET_ONE + + OM_KEY_PREFIX + KEY_SEVEN; + omPutKeyInfo = buildOmKeyInfo(VOL, BUCKET_ONE, KEY_SEVEN, + KEY_SEVEN, KEY_SEVEN_OBJECT_ID, BUCKET_ONE_OBJECT_ID, KEY_SEVEN_SIZE); + keyEvent2 = new OMDBUpdateEvent. + OMUpdateEventBuilder() + .setKey(omPutKey) + .setValue(omPutKeyInfo) + .setTable(omMetadataManager.getKeyTable(getBucketLayout()) + .getName()) + .setAction(OMDBUpdateEvent.OMDBUpdateAction.PUT) + .build(); + + // Test DELETE Event. + // Delete Key1 in Bucket1. + String omDeleteKey = + OM_KEY_PREFIX + VOL + + OM_KEY_PREFIX + BUCKET_ONE + + OM_KEY_PREFIX + KEY_ONE; + OmKeyInfo omDeleteKeyInfo = buildOmKeyInfo(VOL, BUCKET_ONE, KEY_ONE, + KEY_ONE, KEY_ONE_OBJECT_ID, BUCKET_ONE_OBJECT_ID, KEY_ONE_SIZE); + keyEvent3 = new OMDBUpdateEvent. + OMUpdateEventBuilder() + .setKey(omDeleteKey) + .setTable(omMetadataManager.getKeyTable(getBucketLayout()) + .getName()) + .setValue(omDeleteKeyInfo) + .setAction(OMDBUpdateEvent.OMDBUpdateAction.DELETE) + .build(); + + // Test UPDATE Event. + // Resize Key2 in Bucket1. + String omResizeKey = + OM_KEY_PREFIX + VOL + + OM_KEY_PREFIX + BUCKET_ONE + + OM_KEY_PREFIX + KEY_TWO; + OmKeyInfo oldOmResizeKeyInfo = + buildOmKeyInfo(VOL, BUCKET_ONE, KEY_TWO, KEY_TWO, KEY_TWO_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, KEY_TWO_OLD_SIZE); + OmKeyInfo newOmResizeKeyInfo = + buildOmKeyInfo(VOL, BUCKET_ONE, KEY_TWO, KEY_TWO, KEY_TWO_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, KEY_TWO_OLD_SIZE + 100); + keyEvent4 = new OMDBUpdateEvent. + OMUpdateEventBuilder() + .setKey(omResizeKey) + .setOldValue(oldOmResizeKeyInfo) + .setValue(newOmResizeKeyInfo) + .setTable(omMetadataManager.getKeyTable(getBucketLayout()) + .getName()) + .setAction(OMDBUpdateEvent.OMDBUpdateAction.UPDATE) + .build(); + + return new OMUpdateEventBatch( + Arrays.asList(keyEvent1, keyEvent2, keyEvent3, keyEvent4)); + } + + @Test + public void testProcessForCount() throws IOException { + assertNotNull(nsSummaryForBucket1); + assertEquals(3, nsSummaryForBucket1.getNumOfFiles()); + assertNotNull(nsSummaryForBucket2); + assertEquals(3, nsSummaryForBucket2.getNumOfFiles()); + + Set childDirBucket1 = nsSummaryForBucket1.getChildDir(); + assertEquals(0, childDirBucket1.size()); + Set childDirBucket2 = nsSummaryForBucket2.getChildDir(); + assertEquals(0, childDirBucket2.size()); + } + + @Test + public void testProcessForSize() throws IOException { + assertNotNull(nsSummaryForBucket1); + assertEquals( + KEY_THREE_SIZE + KEY_SEVEN_SIZE + KEY_TWO_OLD_SIZE + 100, + nsSummaryForBucket1.getSizeOfFiles()); + assertNotNull(nsSummaryForBucket2); + assertEquals(KEY_FOUR_SIZE + KEY_FIVE_SIZE + KEY_SIX_SIZE, + nsSummaryForBucket2.getSizeOfFiles()); + } + + + @Test + public void testProcessFileBucketSize() { + int[] fileDistBucket1 = nsSummaryForBucket1.getFileSizeBucket(); + int[] fileDistBucket2 = nsSummaryForBucket2.getFileSizeBucket(); + assertEquals(ReconConstants.NUM_OF_FILE_SIZE_BINS, + fileDistBucket1.length); + assertEquals(ReconConstants.NUM_OF_FILE_SIZE_BINS, + fileDistBucket2.length); + + // Check for 1's and 0's in fileDistBucket1 + int[] expectedIndexes1 = {1, 3, 40}; + for (int index = 0; index < fileDistBucket1.length; index++) { + if (contains(expectedIndexes1, index)) { + assertEquals(1, fileDistBucket1[index]); + } else { + assertEquals(0, fileDistBucket1[index]); + } + } + + // Check for 1's and 0's in fileDistBucket2 + int[] expectedIndexes2 = {0, 2, 3}; + for (int index = 0; index < fileDistBucket2.length; index++) { + if (contains(expectedIndexes2, index)) { + assertEquals(1, fileDistBucket2[index]); + } else { + assertEquals(0, fileDistBucket2[index]); + } + } + } + + } + + /** + * Populate OMDB with the following configs. + * vol + * / \ + * bucket1 bucket2 + * / \ \ \ \ + * key1 key2 key3 key4 key5 + * + * @throws IOException + */ + private static void populateOMDB() throws IOException { + writeKeyToOm(reconOMMetadataManager, + KEY_ONE, + BUCKET_ONE, + VOL, + KEY_ONE, + KEY_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + VOL_OBJECT_ID, + KEY_ONE_SIZE, + getBucketLayout()); + writeKeyToOm(reconOMMetadataManager, + KEY_TWO, + BUCKET_ONE, + VOL, + KEY_TWO, + KEY_TWO_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + VOL_OBJECT_ID, + KEY_TWO_OLD_SIZE, + getBucketLayout()); + writeKeyToOm(reconOMMetadataManager, + KEY_THREE, + BUCKET_ONE, + VOL, + KEY_THREE, + KEY_THREE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + BUCKET_ONE_OBJECT_ID, + VOL_OBJECT_ID, + KEY_THREE_SIZE, + getBucketLayout()); + + writeKeyToOm(reconOMMetadataManager, + KEY_FOUR, + BUCKET_TWO, + VOL, + KEY_FOUR, + KEY_FOUR_OBJECT_ID, + BUCKET_TWO_OBJECT_ID, + BUCKET_TWO_OBJECT_ID, + VOL_OBJECT_ID, + KEY_FOUR_SIZE, + getBucketLayout()); + writeKeyToOm(reconOMMetadataManager, + KEY_FIVE, + BUCKET_TWO, + VOL, + KEY_FIVE, + KEY_FIVE_OBJECT_ID, + PARENT_OBJECT_ID_ZERO, + BUCKET_TWO_OBJECT_ID, + VOL_OBJECT_ID, + KEY_FIVE_SIZE, + getBucketLayout()); + } + + /** + * Create a new OM Metadata manager instance with one user, one vol, and two + * buckets. + * + * @throws IOException ioEx + */ + private static void initializeNewOmMetadataManager( + File omDbDir) + throws IOException { + omConfiguration = new OzoneConfiguration(); + omConfiguration.set(OZONE_OM_DB_DIRS, + omDbDir.getAbsolutePath()); + omConfiguration.set(OMConfigKeys + .OZONE_OM_ENABLE_FILESYSTEM_PATHS, "true"); + omMetadataManager = new OmMetadataManagerImpl( + omConfiguration, null); + + String volumeKey = omMetadataManager.getVolumeKey(VOL); + OmVolumeArgs args = + OmVolumeArgs.newBuilder() + .setObjectID(VOL_OBJECT_ID) + .setVolume(VOL) + .setAdminName(TEST_USER) + .setOwnerName(TEST_USER) + .build(); + omMetadataManager.getVolumeTable().put(volumeKey, args); + + OmBucketInfo bucketInfo1 = OmBucketInfo.newBuilder() + .setVolumeName(VOL) + .setBucketName(BUCKET_ONE) + .setObjectID(BUCKET_ONE_OBJECT_ID) + .setBucketLayout(getBucketLayout()) + .build(); + + OmBucketInfo bucketInfo2 = OmBucketInfo.newBuilder() + .setVolumeName(VOL) + .setBucketName(BUCKET_TWO) + .setObjectID(BUCKET_TWO_OBJECT_ID) + .setBucketLayout(getBucketLayout()) + .build(); + + String bucketKey = omMetadataManager.getBucketKey( + bucketInfo1.getVolumeName(), bucketInfo1.getBucketName()); + String bucketKey2 = omMetadataManager.getBucketKey( + bucketInfo2.getVolumeName(), bucketInfo2.getBucketName()); + + omMetadataManager.getBucketTable().put(bucketKey, bucketInfo1); + omMetadataManager.getBucketTable().put(bucketKey2, bucketInfo2); + } + + /** + * Build a key info for put/update action. + * @param volume volume name + * @param bucket bucket name + * @param key key name + * @param fileName file name + * @param objectID object ID + * @param parentObjectId parent object ID + * @param dataSize file size + * @return the KeyInfo + */ + private static OmKeyInfo buildOmKeyInfo(String volume, + String bucket, + String key, + String fileName, + long objectID, + long parentObjectId, + long dataSize) { + return new OmKeyInfo.Builder() + .setBucketName(bucket) + .setVolumeName(volume) + .setKeyName(key) + .setFileName(fileName) + .setReplicationConfig( + StandaloneReplicationConfig.getInstance( + HddsProtos.ReplicationFactor.ONE)) + .setObjectID(objectID) + .setParentObjectID(parentObjectId) + .setDataSize(dataSize) + .build(); + } + + // Helper method to check if an array contains a specific value + private boolean contains(int[] arr, int value) { + for (int num : arr) { + if (num == value) { + return true; + } + } + return false; + } + + private static BucketLayout getBucketLayout() { + return BucketLayout.OBJECT_STORE; + } +}