diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 839a21d55d67..8bec2b9225a3 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -2994,6 +2994,42 @@ public long[] getPerLevelSizeBytes() return compactionStrategyManager.getPerLevelSizeBytes(); } + @Override + public double[] getPerLevelAvgTokenSpace() + { + return compactionStrategyManager.getPerLevelAvgTokenSpace(); + } + + @Override + public double[] getPerLevelMaxDensityThreshold() + { + return compactionStrategyManager.getPerLevelMaxDensityThreshold(); + } + + @Override + public double[] getPerLevelAvgSize() + { + return compactionStrategyManager.getPerLevelAvgSize(); + } + + @Override + public double[] getPerLevelAvgDensity() + { + return compactionStrategyManager.getPerLevelAvgDensity(); + } + + @Override + public double[] getPerLevelAvgDensityMaxDensityThresholdRatio() + { + return compactionStrategyManager.getPerLevelAvgDensityMaxDensityThresholdRatio(); + } + + @Override + public double[] getPerLevelMaxDensityMaxDensityThresholdRatio() + { + return compactionStrategyManager.getPerLevelMaxDensityMaxDensityThresholdRatio(); + } + @Override public boolean isLeveledCompaction() { diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java index 33cd6ebbc25b..91bd7f07e14a 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java @@ -277,6 +277,48 @@ public List importNewSSTables(Set srcPaths, */ public long[] getPerLevelSizeBytes(); + /** + * @return average of sstable covered token spaces in each level. + * null unless unified compaction strategy is used. + * array index corresponds to level(int[0] is for level 0, ...). + */ + public double[] getPerLevelAvgTokenSpace(); + + /** + * @return the maximum density each level is allowed to hold. + * null unless unified compaction strategy is used. + * array index corresponds to level(int[0] is for level 0, ...). + */ + public double[] getPerLevelMaxDensityThreshold(); + + /** + * @return the average size of sstables in each level. + * null unless unified compaction strategy is used. + * array index corresponds to level(int[0] is for level 0, ...). + */ + public double[] getPerLevelAvgSize(); + + /** + * @return the average density of sstables in each level. + * null unless unified compaction strategy is used. + * array index corresponds to level(int[0] is for level 0, ...). + */ + public double[] getPerLevelAvgDensity(); + + /** + * @return the ratio of avg density to the maximum density threshold of that level + * in each level. null unless unified compaction strategy is used. + * array index corresponds to level(int[0] is for level 0, ...). + */ + public double[] getPerLevelAvgDensityMaxDensityThresholdRatio(); + + /** + * @return the ratio of maximum density to the maximum density threshold of that level + * in each level. null unless unified compaction strategy is used. + * array index corresponds to level(int[0] is for level 0, ...). + */ + public double[] getPerLevelMaxDensityMaxDensityThresholdRatio(); + /** * @return true if the table is using LeveledCompactionStrategy. false otherwise. */ diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java index b8eaa5bd812c..4882fca7558b 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java @@ -32,6 +32,8 @@ import java.util.Set; import java.util.TreeMap; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -80,6 +82,7 @@ import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.utils.TimeUUID; +import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy.Level; import static org.apache.cassandra.db.compaction.AbstractStrategyHolder.GroupedSSTableContainer; @@ -690,6 +693,73 @@ public long[] getPerLevelSizeBytes() } } + public double[] getPerLevelAvgTokenSpace() + { + return computeUCSMetric( + data -> { + data.sum[data.levelIndex] += data.sstable.tokenSpaceCoverage(); + data.count[data.levelIndex]++; + }, + CompactionStrategyManager::averageArrayFinalizer + ); + } + + public double[] getPerLevelMaxDensityThreshold() + { + return computeUCSMetric( + data -> { + data.max[data.levelIndex] = Math.max(data.max[data.levelIndex], data.level.max); + }, + CompactionStrategyManager::maxArrayFinalizer + ); + } + + public double[] getPerLevelAvgSize() + { + return computeUCSMetric( + data -> { + data.sum[data.levelIndex] += data.sstable.onDiskLength(); + data.count[data.levelIndex]++; + }, + CompactionStrategyManager::averageArrayFinalizer + ); + } + + public double[] getPerLevelAvgDensity() + { + return computeUCSMetric( + data -> { + data.sum[data.levelIndex] += data.strategy.getDensity(data.sstable); + data.count[data.levelIndex]++; + }, + CompactionStrategyManager::averageArrayFinalizer + ); + } + + public double[] getPerLevelAvgDensityMaxDensityThresholdRatio() + { + double[] avgDensity = getPerLevelAvgDensity(); + if (avgDensity == null) + return null; + + double[] maxThreshold = getPerLevelMaxDensityThreshold(); + double[] res = new double[avgDensity.length]; + for (int i = 0; i < avgDensity.length; i++) + res[i] = avgDensity[i] / maxThreshold[i]; + return res; + } + + public double[] getPerLevelMaxDensityMaxDensityThresholdRatio() + { + return computeUCSMetric( + data -> { + data.sum[data.levelIndex] = Math.max(data.sum[data.levelIndex], data.strategy.getDensity(data.sstable)); + data.max[data.levelIndex] = Math.max(data.max[data.levelIndex], data.level.max); + }, + CompactionStrategyManager::ratioArrayFinalizer + ); + } + public boolean isLeveledCompaction() { readLock.lock(); @@ -702,6 +772,93 @@ public boolean isLeveledCompaction() } } + /** + * Data class for accumulating UCS metrics computation state. + * Holds intermediate values during metric calculation across all strategies and levels. + */ + @VisibleForTesting + static class CompactionStatsMetricsData + { + final double[] sum = new double[UnifiedCompactionStrategy.MAX_LEVELS]; + final int[] count = new int[UnifiedCompactionStrategy.MAX_LEVELS]; + final double[] max = new double[UnifiedCompactionStrategy.MAX_LEVELS]; + int numberOfLevels = 0; + + int levelIndex; + Level level; + SSTableReader sstable; + UnifiedCompactionStrategy strategy; + } + + /** + * Generic helper to compute UCS metrics across all strategies and levels. + * Reduces code duplication for per-level metric calculations. + * + * @param accumulator processes each sstable and updates the metrics data state + * @param finalizer computes the final result array from the accumulated metrics data + * @return computed metric array, one value per level, or null if not using UCS + */ + private double[] computeUCSMetric(Consumer accumulator, Function finalizer) + { + readLock.lock(); + try + { + if (repaired.first() instanceof UnifiedCompactionStrategy) + { + CompactionStatsMetricsData data = new CompactionStatsMetricsData(); + + for (AbstractCompactionStrategy strategy : getAllStrategies()) + { + UnifiedCompactionStrategy ucsStrategy = (UnifiedCompactionStrategy) strategy; + List levels = ucsStrategy.getLevelsSnapshot(); + + data.numberOfLevels = Math.max(data.numberOfLevels, levels.size()); + data.strategy = ucsStrategy; + + for (int i = 0; i < levels.size(); i++) + { + data.levelIndex = i; + data.level = levels.get(i); + for (SSTableReader sstable : levels.get(i).getSSTables()) + { + data.sstable = sstable; + accumulator.accept(data); + } + } + } + + return finalizer.apply(data); + } + return null; + } + finally { + readLock.unlock(); + } + } + + @VisibleForTesting + static double[] averageArrayFinalizer(CompactionStatsMetricsData data) + { + double[] res = new double[data.numberOfLevels]; + for (int i = 0; i < data.numberOfLevels; i++) + res[i] = data.count[i] == 0 ? 0 : data.sum[i] / data.count[i]; + return res; + } + + @VisibleForTesting + static double[] maxArrayFinalizer(CompactionStatsMetricsData data) + { + return Arrays.copyOf(data.max, data.numberOfLevels); + } + + @VisibleForTesting + static double[] ratioArrayFinalizer(CompactionStatsMetricsData data) { + double[] res = new double[data.numberOfLevels]; + for (int i = 0; i < data.numberOfLevels; i++) + res[i] = data.sum[i] / data.max[i]; + return res; + } + public int[] getSSTableCountPerTWCSBucket() { readLock.lock(); diff --git a/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java index d44a9f3288c2..8f6792590e88 100644 --- a/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java @@ -529,6 +529,24 @@ List getLevels() return getLevels(getSSTables(), UnifiedCompactionStrategy::isSuitableForCompaction); } + /** + * @return a list of the levels in the compaction hierarchy, that also includes SSTables that + * are currently undergoing compaction. This is used only for table stats so we can have a consistent + * snapshot of the levels. + */ + List getLevelsSnapshot() + { + Set sstables = getSSTables(); + List suitable = new ArrayList<>(sstables.size()); + for (SSTableReader rdr : sstables) + { + if (isSuitableForCompaction(rdr)) + suitable.add(rdr); + } + + return formLevels(suitable); + } + /** * Groups the sstables passed in into levels. This is used by the strategy to determine * new compactions, and by external tools to analyze the strategy decisions. @@ -546,6 +564,14 @@ public List getLevels(Collection sstables, return formLevels(suitable); } + /** + * Returns the density of the SSTable + */ + public double getDensity(SSTableReader sstable) + { + return shardManager.density(sstable); + } + private List formLevels(List suitable) { maybeUpdateShardManager(); @@ -557,7 +583,7 @@ private List formLevels(List suitable) Level level = new Level(controller, index, 0, maxDensity); for (SSTableReader candidate : suitable) { - final double density = shardManager.density(candidate); + final double density = getDensity(candidate); if (density < level.max) { level.add(candidate); diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTable.java b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTable.java index ec10f18be77d..d8b18c5bee37 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTable.java +++ b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTable.java @@ -29,6 +29,7 @@ public class StatsTable public String tableName; public boolean isIndex; public boolean isLeveledSstable = false; + public boolean isUCSSstable = false; public Object sstableCount; public Object oldSSTableCount; public Long maxSSTableSize; @@ -72,6 +73,12 @@ public class StatsTable public long maximumTombstonesPerSliceLastFiveMinutes; public List sstablesInEachLevel = new ArrayList<>(); public List sstableBytesInEachLevel = new ArrayList<>(); + public List sstableAvgTokenSpaceInEachLevel = new ArrayList<>(); + public List sstableMaxDensityThresholdInEachLevel = new ArrayList<>(); + public List sstableAvgSizeInEachLevel = new ArrayList<>(); + public List sstableAvgDensityInEachLevel = new ArrayList<>(); + public List sstableAvgDensityMaxDensityThresholdRatioInEachLevel = new ArrayList<>(); + public List sstableMaxDensityMaxDensityThresholdRatioInEachLevel = new ArrayList<>(); public int[] sstableCountPerTWCSBucket = null; public Boolean isInCorrectLocation = null; // null: option not active public double droppableTombstoneRatio; diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java index a2005f4b3236..0772a25272d4 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java +++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java @@ -124,6 +124,15 @@ private Map convertStatsTableToMap(StatsTable table) mpTable.put("old_sstable_count", table.oldSSTableCount); mpTable.put("sstables_in_each_level", table.sstablesInEachLevel); mpTable.put("sstable_bytes_in_each_level", table.sstableBytesInEachLevel); + if (table.isUCSSstable) + { + mpTable.put("sstable_avg_token_space_in_each_level", table.sstableAvgTokenSpaceInEachLevel); + mpTable.put("sstable_max_density_threshold_in_each_level", table.sstableMaxDensityThresholdInEachLevel); + mpTable.put("sstable_avg_size_in_each_level", table.sstableAvgSizeInEachLevel); + mpTable.put("sstable_avg_density_in_each_level", table.sstableAvgDensityInEachLevel); + mpTable.put("sstable_avg_density_max_density_threshold_ratio_in_each_level", table.sstableAvgDensityMaxDensityThresholdRatioInEachLevel); + mpTable.put("sstable_max_density_max_density_threshold_ratio_in_each_level", table.sstableMaxDensityMaxDensityThresholdRatioInEachLevel); + } mpTable.put("max_sstable_size", table.maxSSTableSize); mpTable.put("twcs", table.twcs); mpTable.put("space_used_live", table.spaceUsedLive); @@ -275,6 +284,18 @@ private void initializeKeyspaces(NodeProbe probe, boolean ignore, List t } } + addUCSMetric(statsTable, statsTable.sstableAvgTokenSpaceInEachLevel, table.getPerLevelAvgTokenSpace()); + + addUCSMetric(statsTable, statsTable.sstableMaxDensityThresholdInEachLevel, table.getPerLevelMaxDensityThreshold()); + + addUCSMetric(statsTable, statsTable.sstableAvgSizeInEachLevel, table.getPerLevelAvgSize()); + + addUCSMetric(statsTable, statsTable.sstableAvgDensityInEachLevel, table.getPerLevelAvgDensity()); + + addUCSMetric(statsTable, statsTable.sstableAvgDensityMaxDensityThresholdRatioInEachLevel, table.getPerLevelAvgDensityMaxDensityThresholdRatio()); + + addUCSMetric(statsTable, statsTable.sstableMaxDensityMaxDensityThresholdRatioInEachLevel, table.getPerLevelMaxDensityMaxDensityThresholdRatio()); + if (locationCheck) statsTable.isInCorrectLocation = !table.hasMisplacedSSTables(); @@ -447,6 +468,18 @@ private void initializeKeyspaces(NodeProbe probe, boolean ignore, List t } } + private void addUCSMetric(StatsTable statsTable, List acc, double[] values) + { + if (values != null) + { + statsTable.isUCSSstable = true; + for (int level = 0; level < values.length; level++) + { + acc.add(String.format("%.03f", values[level])); + } + } + } + private double getMetricMean(Object metricObject) { if (metricObject instanceof CassandraMetricsRegistry.JmxTimerMBean) { return ((CassandraMetricsRegistry.JmxTimerMBean) metricObject).getMean() / 1000; diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java index 187ca05b8edb..8f378d0530dc 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java +++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java @@ -100,6 +100,22 @@ protected void printStatsTable(StatsTable table, String tableDisplayName, String table.sstableBytesInEachLevel) + "]"); } + if (table.isUCSSstable) + { + out.println(indent + "Average token space for SSTables in each level: [" + String.join(", ", + table.sstableAvgTokenSpaceInEachLevel) + "]"); + out.println(indent + "Maximum density threshold for SSTables in each level: [" + String.join(", ", + table.sstableMaxDensityThresholdInEachLevel) + "]"); + out.println(indent + "Average SSTable size in each level: [" + String.join(", ", + table.sstableAvgSizeInEachLevel) + "]"); + out.println(indent + "Average SSTable density in each level: [" + String.join(", ", + table.sstableAvgDensityInEachLevel) + "]"); + out.println(indent + "Average SSTable density to max threshold ratio in each level: [" + String.join(", ", + table.sstableAvgDensityMaxDensityThresholdRatioInEachLevel) + "]"); + out.println(indent + "Maximum SSTable density to max threshold ratio in each level: [" + String.join(", ", + table.sstableMaxDensityMaxDensityThresholdRatioInEachLevel) + "]"); + } + out.println(indent + "Space used (live): " + table.spaceUsedLive); out.println(indent + "Space used (total): " + table.spaceUsedTotal); out.println(indent + "Space used by snapshots (total): " + table.spaceUsedBySnapshotsTotal); diff --git a/test/distributed/org/apache/cassandra/distributed/test/ColumnFamilyStoreMBeansTest.java b/test/distributed/org/apache/cassandra/distributed/test/ColumnFamilyStoreMBeansTest.java new file mode 100644 index 000000000000..bb5c02686c6f --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/ColumnFamilyStoreMBeansTest.java @@ -0,0 +1,130 @@ +/* + * 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.cassandra.distributed.test; + +import org.junit.*; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.distributed.Cluster; + +import java.io.IOException; +import static org.junit.Assert.assertTrue; + +public class ColumnFamilyStoreMBeansTest extends TestBaseImpl +{ + private static Cluster CLUSTER; + + @BeforeClass + public static void setup() throws IOException + { + CLUSTER = init(Cluster.build(1) + .start()); + + CLUSTER.schemaChange(withKeyspace("DROP KEYSPACE %s")); + CLUSTER.schemaChange(withKeyspace("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}")); + CLUSTER.schemaChange(withKeyspace("CREATE TABLE %s.cf (k text, c1 text, c2 text, PRIMARY KEY (k)) WITH compaction = {'class': 'UnifiedCompactionStrategy', 'scaling_parameters': 'L10'}")); + + for (int i = 0; i < 10000; i++) + CLUSTER.get(1).executeInternal(withKeyspace("INSERT INTO %s.cf (k, c1, c2) VALUES (?, 'value1', 'value2');"), Integer.toString(i)); + + CLUSTER.get(1).nodetool("flush"); + } + + @AfterClass + public static void teardownCluster() throws Exception + { + if (CLUSTER != null) + CLUSTER.close(); + } + + @Test + public void testPerLevelAverageTokenSpace() throws Throwable + { + CLUSTER.get(1).runOnInstance(() -> { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore("cf"); + double[] perLevelAvgTokenSpace = cfs.getPerLevelAvgTokenSpace(); + assertTrue(perLevelAvgTokenSpace.length > 0); + for (int i = 0; i < perLevelAvgTokenSpace.length; i++) + assertTrue(perLevelAvgTokenSpace[i] > 0); + }); + } + + @Test + public void testGetPerLevelMaxDensityThreshold() throws Throwable + { + CLUSTER.get(1).runOnInstance(() -> { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore("cf"); + assertTrue(cfs.getPerLevelMaxDensityThreshold().length > 0); + }); + } + + @Test + public void testGetPerLevelAvgSize() throws Throwable + { + CLUSTER.get(1).runOnInstance(() -> { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore("cf"); + double[] perLevelAvgSize = cfs.getPerLevelAvgSize(); + assertTrue(perLevelAvgSize.length > 0); + for (int i = 0; i < perLevelAvgSize.length; i++) + assertTrue(perLevelAvgSize[i] > 0); + }); + } + + @Test + public void testGetPerLevelAvgDensity() throws Throwable + { + CLUSTER.get(1).runOnInstance(() -> { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore("cf"); + double[] perLevelAvgDensity = cfs.getPerLevelAvgDensity(); + assertTrue(perLevelAvgDensity.length > 0); + for (int i = 0; i < perLevelAvgDensity.length; i++) + assertTrue(perLevelAvgDensity[i] > 0); + }); + } + + @Test + public void testGetPerLevelAvgDensityMaxDensityThresholdRatio() throws Throwable + { + CLUSTER.get(1).runOnInstance(() -> { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore("cf"); + double[] perLevelAvgDensityMaxDensityThresholdRatio = cfs.getPerLevelAvgDensityMaxDensityThresholdRatio(); + assertTrue(perLevelAvgDensityMaxDensityThresholdRatio.length > 0); + for (int i = 0; i < perLevelAvgDensityMaxDensityThresholdRatio.length; i++) + { + assertTrue(0 <= perLevelAvgDensityMaxDensityThresholdRatio[i]); + assertTrue(perLevelAvgDensityMaxDensityThresholdRatio[i] < 1); + } + }); + } + + @Test + public void testGetPerLevelMaxDensityMaxDensityThresholdRatio() throws Throwable + { + CLUSTER.get(1).runOnInstance(() -> { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore("cf"); + double[] perLevelMaxDensityMaxDensityThresholdRatio = cfs.getPerLevelMaxDensityMaxDensityThresholdRatio(); + assertTrue(perLevelMaxDensityMaxDensityThresholdRatio.length > 0); + for (int i = 0; i < perLevelMaxDensityMaxDensityThresholdRatio.length; i++) { + assertTrue(0 <= perLevelMaxDensityMaxDensityThresholdRatio[i]); + assertTrue(perLevelMaxDensityMaxDensityThresholdRatio[i] < 1); + } + }); + } +} diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java index 20420aeef3f5..6e2ad9f1dea5 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java @@ -19,11 +19,7 @@ package org.apache.cassandra.db.compaction; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Set; +import java.util.*; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -433,6 +429,61 @@ public void testCountsByBuckets() Collections.emptyMap()), CompactionStrategyManager.TWCS_BUCKET_COUNT_MAX)); } + @Test + public void testAverageArrayFinalizer() + { + CompactionStrategyManager.CompactionStatsMetricsData data = new CompactionStrategyManager.CompactionStatsMetricsData(); + Random random = new Random(); + data.numberOfLevels = random.nextInt(32); + + for (int i = 0; i < data.numberOfLevels; i++) { + data.sum[i] = random.nextInt(250); + data.count[i] = random.nextInt(10); + } + + double[] res = CompactionStrategyManager.averageArrayFinalizer(data); + assertEquals(res.length, data.numberOfLevels); + for (int i = 0; i < data.numberOfLevels; i++) + if (data.count[i] == 0) + assertEquals(0, res[i], 0.0); + else + assertEquals(data.sum[i] / data.count[i], res[i], 0.1); + } + + @Test + public void testMaxArrayFinalizer() + { + CompactionStrategyManager.CompactionStatsMetricsData data = new CompactionStrategyManager.CompactionStatsMetricsData(); + Random random = new Random(); + data.numberOfLevels = random.nextInt(32); + + for (int i = 0; i < data.numberOfLevels; i++) + data.max[i] = random.nextInt(250); + + double[] res = CompactionStrategyManager.maxArrayFinalizer(data); + assertEquals(res.length, data.numberOfLevels); + for (int i = 0; i < data.numberOfLevels; i++) + assertEquals(data.max[i], res[i], 0.0); + } + + @Test + public void testRatioArrayFinalizer() + { + CompactionStrategyManager.CompactionStatsMetricsData data = new CompactionStrategyManager.CompactionStatsMetricsData(); + Random random = new Random(); + data.numberOfLevels = random.nextInt(32); + + for (int i = 0; i < data.numberOfLevels; i++) { + data.sum[i] = random.nextInt(250); + data.max[i] = 250 + random.nextInt(500); + } + + double[] res = CompactionStrategyManager.ratioArrayFinalizer(data); + assertEquals(res.length, data.numberOfLevels); + for (int i = 0; i < data.numberOfLevels; i++) + assertEquals(data.sum[i] / data.max[i], res[i], 0.1); + } + private MockCFS createJBODMockCFS(int disks) { // Create #disks data directories to simulate JBOD