HDDS-1366. Add ability in Recon to track the number of small files in an Ozone Cluster (#1146)

This commit is contained in:
Shweta Yakkali 2019-08-10 10:14:55 -07:00 committed by Arpit Agarwal
parent e69db45e50
commit d29007fb35
10 changed files with 690 additions and 4 deletions

View File

@ -38,6 +38,9 @@ public class UtilizationSchemaDefinition implements ReconSchemaDefinition {
public static final String CLUSTER_GROWTH_DAILY_TABLE_NAME =
"cluster_growth_daily";
public static final String FILE_COUNT_BY_SIZE_TABLE_NAME =
"file_count_by_size";
@Inject
UtilizationSchemaDefinition(DataSource dataSource) {
this.dataSource = dataSource;
@ -48,6 +51,7 @@ public class UtilizationSchemaDefinition implements ReconSchemaDefinition {
public void initializeSchema() throws SQLException {
Connection conn = dataSource.getConnection();
createClusterGrowthTable(conn);
createFileSizeCount(conn);
}
void createClusterGrowthTable(Connection conn) {
@ -65,5 +69,12 @@ void createClusterGrowthTable(Connection conn) {
.execute();
}
void createFileSizeCount(Connection conn) {
DSL.using(conn).createTableIfNotExists(FILE_COUNT_BY_SIZE_TABLE_NAME)
.column("file_size", SQLDataType.BIGINT)
.column("count", SQLDataType.BIGINT)
.constraint(DSL.constraint("pk_file_size")
.primaryKey("file_size"))
.execute();
}
}

View File

@ -33,9 +33,11 @@
import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
import org.apache.hadoop.ozone.recon.spi.OzoneManagerServiceProvider;
import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTask;
import org.apache.hadoop.ozone.recon.tasks.FileSizeCountTask;
import org.hadoop.ozone.recon.schema.ReconInternalSchemaDefinition;
import org.hadoop.ozone.recon.schema.StatsSchemaDefinition;
import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition;
import org.jooq.Configuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -122,7 +124,7 @@ private void scheduleReconTasks() {
.getInstance(ContainerDBServiceProvider.class);
OzoneManagerServiceProvider ozoneManagerServiceProvider = injector
.getInstance(OzoneManagerServiceProvider.class);
Configuration sqlConfiguration = injector.getInstance(Configuration.class);
long initialDelay = configuration.getTimeDuration(
RECON_OM_SNAPSHOT_TASK_INITIAL_DELAY,
RECON_OM_SNAPSHOT_TASK_INITIAL_DELAY_DEFAULT,
@ -143,6 +145,13 @@ private void scheduleReconTasks() {
ozoneManagerServiceProvider.getOMMetadataManagerInstance());
containerKeyMapperTask.reprocess(
ozoneManagerServiceProvider.getOMMetadataManagerInstance());
FileSizeCountTask fileSizeCountTask = new
FileSizeCountTask(
ozoneManagerServiceProvider.getOMMetadataManagerInstance(),
sqlConfiguration);
fileSizeCountTask.reprocess(
ozoneManagerServiceProvider.getOMMetadataManagerInstance());
} catch (IOException e) {
LOG.error("Unable to get OM " +
"Snapshot", e);

View File

@ -27,7 +27,6 @@
import java.util.TreeMap;
import java.util.stream.Collectors;
import javax.inject.Inject;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
import javax.ws.rs.Path;
@ -38,6 +37,7 @@
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import javax.inject.Inject;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;

View File

@ -0,0 +1,67 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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 javax.inject.Inject;
import org.hadoop.ozone.recon.schema.tables.daos.FileCountBySizeDao;
import org.hadoop.ozone.recon.schema.tables.pojos.FileCountBySize;
import org.jooq.Configuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.ws.rs.GET;
import javax.ws.rs.Path;
import javax.ws.rs.Produces;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.util.List;
/**
* Endpoint for querying the counts of a certain file Size.
*/
@Path("/utilization")
@Produces(MediaType.APPLICATION_JSON)
public class UtilizationService {
private static final Logger LOG =
LoggerFactory.getLogger(UtilizationService.class);
private FileCountBySizeDao fileCountBySizeDao;
@Inject
private Configuration sqlConfiguration;
FileCountBySizeDao getDao() {
if (fileCountBySizeDao == null) {
fileCountBySizeDao = new FileCountBySizeDao(sqlConfiguration);
}
return fileCountBySizeDao;
}
/**
* Return the file counts from Recon DB.
* @return {@link Response}
*/
@GET
@Path("/fileCount")
public Response getFileCounts() {
fileCountBySizeDao = getDao();
List<FileCountBySize> resultSet = fileCountBySizeDao.findAll();
return Response.ok(resultSet).build();
}
}

View File

@ -0,0 +1,255 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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 com.google.inject.Inject;
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.ozone.om.OMMetadataManager;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.utils.db.Table;
import org.apache.hadoop.utils.db.TableIterator;
import org.hadoop.ozone.recon.schema.tables.daos.FileCountBySizeDao;
import org.hadoop.ozone.recon.schema.tables.pojos.FileCountBySize;
import org.jooq.Configuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import static org.apache.hadoop.ozone.recon.tasks.
OMDBUpdateEvent.OMDBUpdateAction.DELETE;
import static org.apache.hadoop.ozone.recon.tasks.
OMDBUpdateEvent.OMDBUpdateAction.PUT;
/**
* Class to iterate over the OM DB and store the counts of existing/new
* files binned into ranges (1KB, 2Kb..,4MB,.., 1TB,..1PB) to the Recon
* fileSize DB.
*/
public class FileSizeCountTask extends ReconDBUpdateTask {
private static final Logger LOG =
LoggerFactory.getLogger(FileSizeCountTask.class);
private int maxBinSize = -1;
private long maxFileSizeUpperBound = 1125899906842624L; // 1 PB
private long[] upperBoundCount;
private long oneKb = 1024L;
private Collection<String> tables = new ArrayList<>();
private FileCountBySizeDao fileCountBySizeDao;
@Inject
public FileSizeCountTask(OMMetadataManager omMetadataManager,
Configuration sqlConfiguration) {
super("FileSizeCountTask");
try {
tables.add(omMetadataManager.getKeyTable().getName());
fileCountBySizeDao = new FileCountBySizeDao(sqlConfiguration);
} catch (Exception e) {
LOG.error("Unable to fetch Key Table updates ", e);
}
upperBoundCount = new long[getMaxBinSize()];
}
long getOneKB() {
return oneKb;
}
long getMaxFileSizeUpperBound() {
return maxFileSizeUpperBound;
}
int getMaxBinSize() {
if (maxBinSize == -1) {
// extra bin to add files > 1PB.
// 1 KB (2 ^ 10) is the smallest tracked file.
maxBinSize = nextClosestPowerIndexOfTwo(maxFileSizeUpperBound) - 10 + 1;
}
return maxBinSize;
}
/**
* Read the Keys from OM snapshot DB and calculate the upper bound of
* File Size it belongs to.
*
* @param omMetadataManager OM Metadata instance.
* @return Pair
*/
@Override
public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
LOG.info("Starting a 'reprocess' run of FileSizeCountTask.");
Table<String, OmKeyInfo> omKeyInfoTable = omMetadataManager.getKeyTable();
try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
keyIter = omKeyInfoTable.iterator()) {
while (keyIter.hasNext()) {
Table.KeyValue<String, OmKeyInfo> kv = keyIter.next();
// reprocess() is a PUT operation on the DB.
updateUpperBoundCount(kv.getValue(), PUT);
}
} catch (IOException ioEx) {
LOG.error("Unable to populate File Size Count in Recon DB. ", ioEx);
return new ImmutablePair<>(getTaskName(), false);
}
populateFileCountBySizeDB();
LOG.info("Completed a 'reprocess' run of FileSizeCountTask.");
return new ImmutablePair<>(getTaskName(), true);
}
@Override
protected Collection<String> getTaskTables() {
return tables;
}
private void updateCountFromDB() {
// Read - Write operations to DB are in ascending order
// of file size upper bounds.
List<FileCountBySize> resultSet = fileCountBySizeDao.findAll();
int index = 0;
if (resultSet != null) {
for (FileCountBySize row : resultSet) {
upperBoundCount[index] = row.getCount();
index++;
}
}
}
/**
* Read the Keys from update events and update the count of files
* pertaining to a certain upper bound.
*
* @param events Update events - PUT/DELETE.
* @return Pair
*/
@Override
Pair<String, Boolean> process(OMUpdateEventBatch events) {
LOG.info("Starting a 'process' run of FileSizeCountTask.");
Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
//update array with file size count from DB
updateCountFromDB();
while (eventIterator.hasNext()) {
OMDBUpdateEvent<String, OmKeyInfo> omdbUpdateEvent = eventIterator.next();
String updatedKey = omdbUpdateEvent.getKey();
OmKeyInfo omKeyInfo = omdbUpdateEvent.getValue();
try{
switch (omdbUpdateEvent.getAction()) {
case PUT:
updateUpperBoundCount(omKeyInfo, PUT);
break;
case DELETE:
updateUpperBoundCount(omKeyInfo, DELETE);
break;
default: LOG.trace("Skipping DB update event : " + omdbUpdateEvent
.getAction());
}
} catch (IOException e) {
LOG.error("Unexpected exception while updating key data : {} {}",
updatedKey, e.getMessage());
return new ImmutablePair<>(getTaskName(), false);
}
populateFileCountBySizeDB();
}
LOG.info("Completed a 'process' run of FileSizeCountTask.");
return new ImmutablePair<>(getTaskName(), true);
}
/**
* Calculate the bin index based on size of the Key.
* index is calculated as the number of right shifts
* needed until dataSize becomes zero.
*
* @param dataSize Size of the key.
* @return int bin index in upperBoundCount
*/
public int calculateBinIndex(long dataSize) {
if (dataSize >= getMaxFileSizeUpperBound()) {
return getMaxBinSize() - 1;
}
int index = nextClosestPowerIndexOfTwo(dataSize);
// The smallest file size being tracked for count
// is 1 KB i.e. 1024 = 2 ^ 10.
return index < 10 ? 0 : index - 10;
}
int nextClosestPowerIndexOfTwo(long dataSize) {
int index = 0;
while(dataSize != 0) {
dataSize >>= 1;
index += 1;
}
return index;
}
/**
* Populate DB with the counts of file sizes calculated
* using the dao.
*
*/
void populateFileCountBySizeDB() {
for (int i = 0; i < upperBoundCount.length; i++) {
long fileSizeUpperBound = (i == upperBoundCount.length - 1) ?
Long.MAX_VALUE : (long) Math.pow(2, (10 + i));
FileCountBySize fileCountRecord =
fileCountBySizeDao.findById(fileSizeUpperBound);
FileCountBySize newRecord = new
FileCountBySize(fileSizeUpperBound, upperBoundCount[i]);
if (fileCountRecord == null) {
fileCountBySizeDao.insert(newRecord);
} else {
fileCountBySizeDao.update(newRecord);
}
}
}
/**
* Calculate and update the count of files being tracked by
* upperBoundCount[].
* Used by reprocess() and process().
*
* @param omKeyInfo OmKey being updated for count
* @param operation (PUT, DELETE)
*/
void updateUpperBoundCount(OmKeyInfo omKeyInfo,
OMDBUpdateEvent.OMDBUpdateAction operation) throws IOException {
int binIndex = calculateBinIndex(omKeyInfo.getDataSize());
if (operation == PUT) {
upperBoundCount[binIndex]++;
} else if (operation == DELETE) {
if (upperBoundCount[binIndex] != 0) {
//decrement only if it had files before, default DB value is 0
upperBoundCount[binIndex]--;
} else {
LOG.debug("Cannot decrement count. Default value is 0 (zero).");
throw new IOException("Cannot decrement count. "
+ "Default value is 0 (zero).");
}
}
}
}

View File

@ -160,6 +160,34 @@ protected void writeDataToOm(OMMetadataManager omMetadataManager,
.build());
}
/**
* Write a key to OM instance.
* @throws IOException while writing.
*/
protected void writeDataToOm(OMMetadataManager omMetadataManager,
String key,
String bucket,
String volume,
Long dataSize,
List<OmKeyLocationInfoGroup>
omKeyLocationInfoGroupList)
throws IOException {
String omKey = omMetadataManager.getOzoneKey(volume,
bucket, key);
omMetadataManager.getKeyTable().put(omKey,
new OmKeyInfo.Builder()
.setBucketName(bucket)
.setVolumeName(volume)
.setKeyName(key)
.setDataSize(dataSize)
.setReplicationFactor(HddsProtos.ReplicationFactor.ONE)
.setReplicationType(HddsProtos.ReplicationType.STAND_ALONE)
.setOmKeyLocationInfos(omKeyLocationInfoGroupList)
.build());
}
/**
* Return random pipeline.
* @return pipeline

View File

@ -0,0 +1,86 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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 org.apache.hadoop.ozone.recon.ReconUtils;
import org.hadoop.ozone.recon.schema.tables.daos.FileCountBySizeDao;
import org.hadoop.ozone.recon.schema.tables.pojos.FileCountBySize;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import javax.ws.rs.core.Response;
import java.util.ArrayList;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.powermock.api.mockito.PowerMockito.mock;
import static org.powermock.api.mockito.PowerMockito.when;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
/**
* Test for File size count service.
*/
@RunWith(PowerMockRunner.class)
@PowerMockIgnore({"javax.management.*", "javax.net.ssl.*"})
@PrepareForTest(ReconUtils.class)
public class TestUtilizationService {
private UtilizationService utilizationService;
@Mock private FileCountBySizeDao fileCountBySizeDao;
private int maxBinSize = 42;
private List<FileCountBySize> setUpResultList() {
List<FileCountBySize> resultList = new ArrayList<>();
for (int i = 0; i < maxBinSize; i++) {
if (i == maxBinSize - 1) {
// for last bin file count is 41.
resultList.add(new FileCountBySize(Long.MAX_VALUE, (long) i));
} else {
// count of files of upperBound is equal to it's index.
resultList.add(new FileCountBySize((long) Math.pow(2, (10+i)),
(long) i));
}
}
return resultList;
}
@Test
public void testGetFileCounts() {
List<FileCountBySize> resultList = setUpResultList();
utilizationService = mock(UtilizationService.class);
when(utilizationService.getFileCounts()).thenCallRealMethod();
when(utilizationService.getDao()).thenReturn(fileCountBySizeDao);
when(fileCountBySizeDao.findAll()).thenReturn(resultList);
Response response = utilizationService.getFileCounts();
// get result list from Response entity
List<FileCountBySize> responseList =
(List<FileCountBySize>) response.getEntity();
verify(fileCountBySizeDao, times(1)).findAll();
assertEquals(maxBinSize, responseList.size());
assertEquals(resultList, responseList);
}
}

View File

@ -18,11 +18,14 @@
package org.apache.hadoop.ozone.recon.persistence;
import static org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition.CLUSTER_GROWTH_DAILY_TABLE_NAME;
import static org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition.FILE_COUNT_BY_SIZE_TABLE_NAME;
import static org.hadoop.ozone.recon.schema.tables.ClusterGrowthDailyTable.CLUSTER_GROWTH_DAILY;
import static org.junit.Assert.assertEquals;
import java.sql.Connection;
import java.sql.DatabaseMetaData;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Timestamp;
import java.sql.Types;
import java.util.ArrayList;
@ -34,8 +37,13 @@
import org.apache.commons.lang3.tuple.Pair;
import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition;
import org.hadoop.ozone.recon.schema.tables.daos.ClusterGrowthDailyDao;
import org.hadoop.ozone.recon.schema.tables.daos.FileCountBySizeDao;
import org.hadoop.ozone.recon.schema.tables.pojos.ClusterGrowthDaily;
import org.hadoop.ozone.recon.schema.tables.pojos.FileCountBySize;
import org.hadoop.ozone.recon.schema.tables.records.FileCountBySizeRecord;
import org.jooq.Configuration;
import org.jooq.Table;
import org.jooq.UniqueKey;
import org.junit.Assert;
import org.junit.Test;
@ -78,6 +86,26 @@ public void testReconSchemaCreated() throws Exception {
Assert.assertEquals(8, actualPairs.size());
Assert.assertEquals(expectedPairs, actualPairs);
ResultSet resultSetFileCount = metaData.getColumns(null, null,
FILE_COUNT_BY_SIZE_TABLE_NAME, null);
List<Pair<String, Integer>> expectedPairsFileCount = new ArrayList<>();
expectedPairsFileCount.add(
new ImmutablePair<>("file_size", Types.INTEGER));
expectedPairsFileCount.add(
new ImmutablePair<>("count", Types.INTEGER));
List<Pair<String, Integer>> actualPairsFileCount = new ArrayList<>();
while(resultSetFileCount.next()) {
actualPairsFileCount.add(new ImmutablePair<>(resultSetFileCount.getString(
"COLUMN_NAME"), resultSetFileCount.getInt(
"DATA_TYPE")));
}
assertEquals("Unexpected number of columns",
2, actualPairsFileCount.size());
assertEquals("Columns Do not Match ",
expectedPairsFileCount, actualPairsFileCount);
}
@Test
@ -85,7 +113,6 @@ public void testClusterGrowthDailyCRUDOperations() throws Exception {
// Verify table exists
UtilizationSchemaDefinition schemaDefinition = getInjector().getInstance(
UtilizationSchemaDefinition.class);
schemaDefinition.initializeSchema();
DataSource ds = getInjector().getInstance(DataSource.class);
@ -157,4 +184,51 @@ public void testClusterGrowthDailyCRUDOperations() throws Exception {
Assert.assertNull(dbRecord);
}
@Test
public void testFileCountBySizeCRUDOperations() throws SQLException {
UtilizationSchemaDefinition schemaDefinition = getInjector().getInstance(
UtilizationSchemaDefinition.class);
schemaDefinition.initializeSchema();
DataSource ds = getInjector().getInstance(DataSource.class);
Connection connection = ds.getConnection();
DatabaseMetaData metaData = connection.getMetaData();
ResultSet resultSet = metaData.getTables(null, null,
FILE_COUNT_BY_SIZE_TABLE_NAME, null);
while (resultSet.next()) {
Assert.assertEquals(FILE_COUNT_BY_SIZE_TABLE_NAME,
resultSet.getString("TABLE_NAME"));
}
FileCountBySizeDao fileCountBySizeDao = new FileCountBySizeDao(
getInjector().getInstance(Configuration.class));
FileCountBySize newRecord = new FileCountBySize();
newRecord.setFileSize(1024L);
newRecord.setCount(1L);
fileCountBySizeDao.insert(newRecord);
FileCountBySize dbRecord = fileCountBySizeDao.findById(1024L);
assertEquals(Long.valueOf(1), dbRecord.getCount());
dbRecord.setCount(2L);
fileCountBySizeDao.update(dbRecord);
dbRecord = fileCountBySizeDao.findById(1024L);
assertEquals(Long.valueOf(2), dbRecord.getCount());
Table<FileCountBySizeRecord> fileCountBySizeRecordTable =
fileCountBySizeDao.getTable();
List<UniqueKey<FileCountBySizeRecord>> tableKeys =
fileCountBySizeRecordTable.getKeys();
for (UniqueKey key : tableKeys) {
String name = key.getName();
}
}
}

View File

@ -0,0 +1,140 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.ozone.om.OMMetadataManager;
import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.utils.db.TypedTable;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import java.io.IOException;
import static org.apache.hadoop.ozone.recon.tasks.
OMDBUpdateEvent.OMDBUpdateAction.PUT;
import static org.junit.Assert.assertEquals;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.BDDMockito.given;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.times;
import static org.powermock.api.mockito.PowerMockito.mock;
import static org.powermock.api.mockito.PowerMockito.when;
/**
* Unit test for File Size Count Task.
*/
@RunWith(PowerMockRunner.class)
@PowerMockIgnore({"javax.management.*", "javax.net.ssl.*"})
@PrepareForTest(OmKeyInfo.class)
public class TestFileSizeCountTask {
@Test
public void testCalculateBinIndex() {
FileSizeCountTask fileSizeCountTask = mock(FileSizeCountTask.class);
when(fileSizeCountTask.getMaxFileSizeUpperBound()).
thenReturn(1125899906842624L); // 1 PB
when(fileSizeCountTask.getOneKB()).thenReturn(1024L);
when(fileSizeCountTask.getMaxBinSize()).thenReturn(42);
when(fileSizeCountTask.calculateBinIndex(anyLong())).thenCallRealMethod();
when(fileSizeCountTask.nextClosestPowerIndexOfTwo(
anyLong())).thenCallRealMethod();
long fileSize = 1024L; // 1 KB
int binIndex = fileSizeCountTask.calculateBinIndex(fileSize);
assertEquals(1, binIndex);
fileSize = 1023L; // 1KB - 1B
binIndex = fileSizeCountTask.calculateBinIndex(fileSize);
assertEquals(0, binIndex);
fileSize = 562949953421312L; // 512 TB
binIndex = fileSizeCountTask.calculateBinIndex(fileSize);
assertEquals(40, binIndex);
fileSize = 562949953421313L; // (512 TB + 1B)
binIndex = fileSizeCountTask.calculateBinIndex(fileSize);
assertEquals(40, binIndex);
fileSize = 562949953421311L; // (512 TB - 1B)
binIndex = fileSizeCountTask.calculateBinIndex(fileSize);
assertEquals(39, binIndex);
fileSize = 1125899906842624L; // 1 PB - last (extra) bin
binIndex = fileSizeCountTask.calculateBinIndex(fileSize);
assertEquals(41, binIndex);
fileSize = 100000L;
binIndex = fileSizeCountTask.calculateBinIndex(fileSize);
assertEquals(7, binIndex);
fileSize = 1125899906842623L; // (1 PB - 1B)
binIndex = fileSizeCountTask.calculateBinIndex(fileSize);
assertEquals(40, binIndex);
fileSize = 1125899906842624L * 4; // 4 PB - last extra bin
binIndex = fileSizeCountTask.calculateBinIndex(fileSize);
assertEquals(41, binIndex);
fileSize = Long.MAX_VALUE; // extra bin
binIndex = fileSizeCountTask.calculateBinIndex(fileSize);
assertEquals(41, binIndex);
}
@Test
public void testFileCountBySizeReprocess() throws IOException {
OmKeyInfo omKeyInfo1 = mock(OmKeyInfo.class);
given(omKeyInfo1.getKeyName()).willReturn("key1");
given(omKeyInfo1.getDataSize()).willReturn(1000L);
OMMetadataManager omMetadataManager = mock(OmMetadataManagerImpl.class);
TypedTable<String, OmKeyInfo> keyTable = mock(TypedTable.class);
TypedTable.TypedTableIterator mockKeyIter = mock(TypedTable
.TypedTableIterator.class);
TypedTable.TypedKeyValue mockKeyValue = mock(
TypedTable.TypedKeyValue.class);
when(keyTable.iterator()).thenReturn(mockKeyIter);
when(omMetadataManager.getKeyTable()).thenReturn(keyTable);
when(mockKeyIter.hasNext()).thenReturn(true).thenReturn(false);
when(mockKeyIter.next()).thenReturn(mockKeyValue);
when(mockKeyValue.getValue()).thenReturn(omKeyInfo1);
FileSizeCountTask fileSizeCountTask = mock(FileSizeCountTask.class);
when(fileSizeCountTask.getMaxFileSizeUpperBound()).
thenReturn(4096L);
when(fileSizeCountTask.getOneKB()).thenReturn(1024L);
when(fileSizeCountTask.reprocess(omMetadataManager)).thenCallRealMethod();
//call reprocess()
fileSizeCountTask.reprocess(omMetadataManager);
verify(fileSizeCountTask, times(1)).
updateUpperBoundCount(omKeyInfo1, PUT);
verify(fileSizeCountTask,
times(1)).populateFileCountBySizeDB();
}
}

View File

@ -0,0 +1,16 @@
# 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.
mock-maker-inline