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..4c7277e866f6 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java @@ -32,6 +32,7 @@ import java.util.Set; import java.util.TreeMap; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.BiFunction; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -80,6 +81,8 @@ import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.utils.TimeUUID; +import org.apache.cassandra.utils.TriFunction; +import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy.Level; import static org.apache.cassandra.db.compaction.AbstractStrategyHolder.GroupedSSTableContainer; @@ -690,6 +693,94 @@ public long[] getPerLevelSizeBytes() } } + public double[] getPerLevelAvgTokenSpace() + { + return avgUCSHelper((ucs, sstable) -> + sstable.tokenSpaceCoverage()); + } + + public double[] getPerLevelMaxDensityThreshold() + { + return perLevelUCSHelper((level, sstable, acc) -> Math.max(acc, level.max)); + } + + public double[] getPerLevelAvgSize() + { + return avgUCSHelper((ucs, sstable) -> (double) sstable.onDiskLength()); + } + + public double[] getPerLevelAvgDensity() + { + return avgUCSHelper(UnifiedCompactionStrategy::getDensity); + } + + public double[] getPerLevelAvgDensityMaxDensityThresholdRatio() + { + readLock.lock(); + try + { + if (repaired.first() instanceof UnifiedCompactionStrategy) + { + double[] avgDensityPerLevel = getPerLevelAvgDensity(); + double[] maxDensityThresholdPerLevel = getPerLevelMaxDensityThreshold(); + + double[] res = new double[avgDensityPerLevel.length]; + + for (int i = 0; i < avgDensityPerLevel.length; i++) + { + res[i] = avgDensityPerLevel[i] / maxDensityThresholdPerLevel[i]; + } + + return res; + } + return null; + } + finally + { + readLock.unlock(); + } + } + + public double[] getPerLevelMaxDensityMaxDensityThresholdRatio() + { + readLock.lock(); + try + { + if (repaired.first() instanceof UnifiedCompactionStrategy) + { + + double[] maxDensityThresholdPerLevel = getPerLevelMaxDensityThreshold(); + double[] maxDensityPerLevel = new double[maxDensityThresholdPerLevel.length]; + double[] res = new double[maxDensityThresholdPerLevel.length]; + + for (AbstractCompactionStrategy strategy : getAllStrategies()) + { + UnifiedCompactionStrategy ucsStrategy = (UnifiedCompactionStrategy) strategy; + List levels = ucsStrategy.getLevelsSnapshot(); + for (int i = 0; i < levels.size(); i++) + { + for (SSTableReader sstable : levels.get(i).getSSTables()) + { + maxDensityPerLevel[i] = Math.max(maxDensityPerLevel[i], ucsStrategy.getDensity(sstable)); + } + } + } + + for (int i = 0; i < maxDensityThresholdPerLevel.length; i++) + { + res[i] = maxDensityPerLevel[i] / maxDensityThresholdPerLevel[i]; + } + + return res; + } + return null; + } + finally + { + readLock.unlock(); + } + } + public boolean isLeveledCompaction() { readLock.lock(); @@ -702,6 +793,85 @@ public boolean isLeveledCompaction() } } + double[] avgUCSHelper(BiFunction fn) + { + readLock.lock(); + try + { + if (repaired.first() instanceof UnifiedCompactionStrategy) + { + int numberOfLevels = 0; + + double[] sum = new double[UnifiedCompactionStrategy.MAX_LEVELS]; + int[] count = new int[UnifiedCompactionStrategy.MAX_LEVELS]; + + for (AbstractCompactionStrategy strategy : getAllStrategies()) + { + UnifiedCompactionStrategy ucsStrategy = (UnifiedCompactionStrategy) strategy; + List levels = ucsStrategy.getLevelsSnapshot(); + numberOfLevels = Math.max(numberOfLevels, levels.size()); + for (int i = 0; i < levels.size(); i++) + { + for (SSTableReader sstable : levels.get(i).getSSTables()) + { + sum[i] += fn.apply(ucsStrategy, sstable); + count[i] += 1; + } + } + } + + double[] res = new double[numberOfLevels]; + for (int i = 0; i < numberOfLevels; i++) + res[i] = count[i] == 0 ? 0 : sum[i] / count[i]; + + return res; + } + return null; + } + finally + { + readLock.unlock(); + } + } + + double[] perLevelUCSHelper(TriFunction fn) + { + readLock.lock(); + try + { + if (repaired.first() instanceof UnifiedCompactionStrategy) + { + int numberOfLevels = 0; + + double[] tmp = new double[UnifiedCompactionStrategy.MAX_LEVELS]; + + for (AbstractCompactionStrategy strategy : getAllStrategies()) + { + UnifiedCompactionStrategy ucsStrategy = (UnifiedCompactionStrategy) strategy; + List levels = ucsStrategy.getLevelsSnapshot(); + numberOfLevels = Math.max(numberOfLevels, levels.size()); + for (int i = 0; i < levels.size(); i++) + { + for (SSTableReader sstable : levels.get(i).getSSTables()) + { + tmp[i] = fn.apply(levels.get(i), sstable, tmp[i]); + } + } + } + + double[] res = new double[numberOfLevels]; + System.arraycopy(tmp, 0, res, 0, numberOfLevels); + + return res; + } + return null; + } + finally + { + readLock.unlock(); + } + } + 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..6f9332f52f25 100644 --- a/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java @@ -529,6 +529,25 @@ 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. + */ + @VisibleForTesting + 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 +565,11 @@ public List getLevels(Collection sstables, return formLevels(suitable); } + public double getDensity(SSTableReader sstable) + { + return shardManager.density(sstable); + } + private List formLevels(List suitable) { maybeUpdateShardManager(); @@ -557,7 +581,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..580ac1ae5834 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,66 @@ private void initializeKeyspaces(NodeProbe probe, boolean ignore, List t } } + double[] ucsSSTableTokenSpace = table.getPerLevelAvgTokenSpace(); + if (ucsSSTableTokenSpace != null) + { + statsTable.isUCSSstable = true; + for (int level = 0; level < ucsSSTableTokenSpace.length; level++) + { + statsTable.sstableAvgTokenSpaceInEachLevel.add(String.format("%.03f", ucsSSTableTokenSpace[level])); + } + } + + double[] ucsMaxDensityThreshold = table.getPerLevelMaxDensityThreshold(); + if (ucsMaxDensityThreshold != null) + { + statsTable.isUCSSstable = true; + for (int level = 0; level < ucsMaxDensityThreshold.length; level++) + { + statsTable.sstableMaxDensityThresholdInEachLevel.add(String.format("%.03f", ucsMaxDensityThreshold[level])); + } + } + + double[] ucsSsTableAvgSize = table.getPerLevelAvgSize(); + if (ucsSsTableAvgSize != null) + { + statsTable.isUCSSstable = true; + for (int level = 0; level < ucsSsTableAvgSize.length; level++) + { + statsTable.sstableAvgSizeInEachLevel.add(String.format("%.03f", ucsSsTableAvgSize[level])); + } + } + + double[] ucsSStableAvgDensity = table.getPerLevelAvgDensity(); + if (ucsSStableAvgDensity != null) + { + statsTable.isUCSSstable = true; + for (int level = 0; level < ucsSStableAvgDensity.length; level++) + { + statsTable.sstableAvgDensityInEachLevel.add(String.format("%.03f", ucsSStableAvgDensity[level])); + } + } + + double[] ucsSStableAvgDensityMaxDensityThresholdRatio = table.getPerLevelAvgDensityMaxDensityThresholdRatio(); + if (ucsSStableAvgDensityMaxDensityThresholdRatio != null) + { + statsTable.isUCSSstable = true; + for (int level = 0; level < ucsSStableAvgDensityMaxDensityThresholdRatio.length; level++) + { + statsTable.sstableAvgDensityMaxDensityThresholdRatioInEachLevel.add(String.format("%.03f", ucsSStableAvgDensityMaxDensityThresholdRatio[level])); + } + } + + double[] ucsSStableMaxDensityMaxDensityThresholdRatioInEachLevel = table.getPerLevelMaxDensityMaxDensityThresholdRatio(); + if (ucsSStableMaxDensityMaxDensityThresholdRatioInEachLevel != null) + { + statsTable.isUCSSstable = true; + for (int level = 0; level < ucsSStableMaxDensityMaxDensityThresholdRatioInEachLevel.length; level++) + { + statsTable.sstableMaxDensityMaxDensityThresholdRatioInEachLevel.add(String.format("%.03f", ucsSStableMaxDensityMaxDensityThresholdRatioInEachLevel[level])); + } + } + if (locationCheck) statsTable.isInCorrectLocation = !table.hasMisplacedSSTables(); 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..24ccd94bf6cd 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 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 + "Ratio of average SSTable density and maximum density threshold in each level: [" + String.join(", ", + table.sstableAvgDensityMaxDensityThresholdRatioInEachLevel) + "]"); + out.println(indent + "Ratio of maximum SSTable density and maximum density threshold 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..59e79ca33d14 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/ColumnFamilyStoreMBeansTest.java @@ -0,0 +1,126 @@ +/* + * 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.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; + +public class ColumnFamilyStoreMBeansTest extends TestBaseImpl +{ + private static Cluster CLUSTER; + + @BeforeClass + public static void setup() throws IOException + { + CLUSTER = init(Cluster.build(1).withConfig(c -> + c.with(GOSSIP, NETWORK)) + .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"); + } + + @Test + public void testPerLevelAverageTokenSpace() throws Throwable + { + CLUSTER.get(1).runOnInstance(() -> { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore("cf"); + double[] perLevelAvgTokenSpace = cfs.getPerLevelAvgTokenSpace(); + assert(perLevelAvgTokenSpace.length > 0); + for (int i = 0; i < perLevelAvgTokenSpace.length; i++) + Assert.assertTrue(perLevelAvgTokenSpace[i] > 0); + }); + } + + @Test + public void testGetPerLevelMaxDensityThreshold() throws Throwable + { + CLUSTER.get(1).runOnInstance(() -> { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore("cf"); + Assert.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(); + assert(perLevelAvgSize.length > 0); + for (int i = 0; i < perLevelAvgSize.length; i++) + Assert.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(); + assert(perLevelAvgDensity.length > 0); + for (int i = 0; i < perLevelAvgDensity.length; i++) + Assert.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(); + assert(perLevelAvgDensityMaxDensityThresholdRatio.length > 0); + for (int i = 0; i < perLevelAvgDensityMaxDensityThresholdRatio.length; i++) + { + Assert.assertTrue(0 <= perLevelAvgDensityMaxDensityThresholdRatio[i]); + Assert.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(); + assert(perLevelMaxDensityMaxDensityThresholdRatio.length > 0); + for (int i = 0; i < perLevelMaxDensityMaxDensityThresholdRatio.length; i++) { + Assert.assertTrue(0 <= perLevelMaxDensityMaxDensityThresholdRatio[i]); + Assert.assertTrue(perLevelMaxDensityMaxDensityThresholdRatio[i] < 1); + } + }); + } +} \ No newline at end of file