diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractSeriesAggregationScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractSeriesAggregationScanOperator.java index 1a289ef1fd76..3c0b5ce82d87 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractSeriesAggregationScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractSeriesAggregationScanOperator.java @@ -35,6 +35,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Optional; import java.util.concurrent.TimeUnit; import static org.apache.iotdb.db.queryengine.execution.operator.AggregationUtil.appendAggregationResult; @@ -140,7 +141,11 @@ public TsBlock next() throws Exception { // calculate aggregation result on current time window // Keep curTimeRange if the calculation of this timeRange is not done - if (calculateAggregationResultForCurrentTimeRange()) { + Optional b = calculateAggregationResultForCurrentTimeRange(); + if (!b.isPresent()) { + continue; + } + if (b.get()) { curTimeRange = null; } } @@ -164,41 +169,58 @@ public boolean isFinished() throws Exception { @SuppressWarnings("squid:S112") /** Return true if we have the result of this timeRange. */ - protected boolean calculateAggregationResultForCurrentTimeRange() { + protected Optional calculateAggregationResultForCurrentTimeRange() { try { if (calcFromCachedData()) { updateResultTsBlock(); - return true; + return Optional.of(true); } if (readAndCalcFromPage()) { updateResultTsBlock(); - return true; + return Optional.of(true); } // only when all the page data has been consumed, we need to read the chunk data if (!seriesScanUtil.hasNextPage() && readAndCalcFromChunk()) { updateResultTsBlock(); - return true; + return Optional.of(true); } // only when all the page and chunk data has been consumed, we need to read the file data - if (!seriesScanUtil.hasNextPage() - && !seriesScanUtil.hasNextChunk() - && readAndCalcFromFile()) { - updateResultTsBlock(); - return true; + Optional b; + if (!seriesScanUtil.hasNextPage()) { + b = seriesScanUtil.hasNextChunk(); + if (!b.isPresent()) { + return b; + } + if (!b.get() && readAndCalcFromFile()) { + updateResultTsBlock(); + return Optional.of(true); + } } // If the TimeRange is (Long.MIN_VALUE, Long.MAX_VALUE), for Aggregators like countAggregator, // we have to consume all the data before we finish the aggregation calculation. - if (seriesScanUtil.hasNextPage() - || seriesScanUtil.hasNextChunk() - || seriesScanUtil.hasNextFile()) { - return false; + if (seriesScanUtil.hasNextPage()) { + return Optional.of(false); + } + b = seriesScanUtil.hasNextChunk(); + if (!b.isPresent()) { + return b; + } + if (b.get()) { + return Optional.of(false); + } + b = seriesScanUtil.hasNextFile(); + if (!b.isPresent()) { + return b; + } + if (b.get()) { + return Optional.of(false); } updateResultTsBlock(); - return true; + return Optional.of(true); } catch (IOException e) { throw new RuntimeException("Error while scanning the file", e); } @@ -241,7 +263,14 @@ protected void calcFromStatistics(Statistics timeStatistics, Statistics[] valueS protected boolean readAndCalcFromFile() throws IOException { // start stopwatch long start = System.nanoTime(); - while (System.nanoTime() - start < leftRuntimeOfOneNextCall && seriesScanUtil.hasNextFile()) { + while (System.nanoTime() - start < leftRuntimeOfOneNextCall) { + Optional b = seriesScanUtil.hasNextFile(); + if (!b.isPresent()) { + continue; + } + if (!b.get()) { + break; + } if (canUseStatistics && seriesScanUtil.canUseCurrentFileStatistics()) { Statistics fileTimeStatistics = seriesScanUtil.currentFileTimeStatistics(); if (fileTimeStatistics.getStartTime() > curTimeRange.getMax()) { @@ -282,7 +311,14 @@ protected boolean readAndCalcFromFile() throws IOException { protected boolean readAndCalcFromChunk() throws IOException { // start stopwatch long start = System.nanoTime(); - while (System.nanoTime() - start < leftRuntimeOfOneNextCall && seriesScanUtil.hasNextChunk()) { + while (System.nanoTime() - start < leftRuntimeOfOneNextCall) { + Optional b = seriesScanUtil.hasNextChunk(); + if (!b.isPresent()) { + continue; + } + if (!b.get()) { + break; + } if (canUseStatistics && seriesScanUtil.canUseCurrentChunkStatistics()) { Statistics chunkTimeStatistics = seriesScanUtil.currentChunkTimeStatistics(); if (chunkTimeStatistics.getStartTime() > curTimeRange.getMax()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractSeriesScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractSeriesScanOperator.java index c03764bd61cb..cad688be1851 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractSeriesScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractSeriesScanOperator.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.util.List; +import java.util.Optional; import java.util.concurrent.TimeUnit; public abstract class AbstractSeriesScanOperator extends AbstractDataSourceOperator { @@ -70,10 +71,19 @@ public boolean hasNext() throws Exception { * 2. consume chunk data secondly * 3. consume next file finally */ - if (!readPageData() && !readChunkData() && !readFileData()) { - noMoreData = true; - break; + if (readPageData()) { + continue; } + Optional b = readChunkData(); + if (!b.isPresent() || b.get()) { + continue; + } + b = readFileData(); + if (!b.isPresent() || b.get()) { + continue; + } + noMoreData = true; + break; } while (System.nanoTime() - start < maxRuntime && !resultTsBlockBuilder.isFull() @@ -87,22 +97,28 @@ public boolean hasNext() throws Exception { } } - protected boolean readFileData() throws IOException { - while (seriesScanUtil.hasNextFile()) { - if (readChunkData()) { - return true; - } + protected Optional readFileData() throws IOException { + Optional b = seriesScanUtil.hasNextFile(); + if (!b.isPresent() || !b.get()) { + return b; } - return false; + b = readChunkData(); + if (!b.isPresent() || b.get()) { + return b; + } + return Optional.empty(); } - protected boolean readChunkData() throws IOException { - while (seriesScanUtil.hasNextChunk()) { - if (readPageData()) { - return true; - } + protected Optional readChunkData() throws IOException { + Optional b = seriesScanUtil.hasNextChunk(); + if (!b.isPresent() || !b.get()) { + return b; } - return false; + + if (readPageData()) { + return Optional.of(true); + } + return Optional.empty(); } protected boolean readPageData() throws IOException { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java index b9b500ad83e9..817649869d94 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/SeriesScanUtil.java @@ -219,9 +219,14 @@ protected DescPriorityMergeReader getDescPriorityMergeReader() { // file level methods ///////////////////////////////////////////////////////////////////////////////////////////////// - public boolean hasNextFile() throws IOException { + // When Optional.empty() is returned, it means that the current hasNextFile has not been fully + // executed. In order to avoid the execution time of this method exceeding the allocated time + // slice, it is return early in this way. For the upper-level method, when encountering + // Optional.empty(), it needs to return directly to the checkpoint method that checks the operator + // execution time slice. + public Optional hasNextFile() throws IOException { if (!paginationController.hasCurLimit()) { - return false; + return Optional.of(false); } if (!unSeqPageReaders.isEmpty() @@ -241,21 +246,25 @@ public boolean hasNextFile() throws IOException { } if (firstTimeSeriesMetadata != null) { - return true; + return Optional.of(true); } - while (firstTimeSeriesMetadata == null - && (orderUtils.hasNextSeqResource() - || orderUtils.hasNextUnseqResource() - || !seqTimeSeriesMetadata.isEmpty() - || !unSeqTimeSeriesMetadata.isEmpty())) { + boolean checked = false; + if (orderUtils.hasNextSeqResource() + || orderUtils.hasNextUnseqResource() + || !seqTimeSeriesMetadata.isEmpty() + || !unSeqTimeSeriesMetadata.isEmpty()) { // init first time series metadata whose startTime is minimum tryToUnpackAllOverlappedFilesToTimeSeriesMetadata(); // filter file based on push-down conditions filterFirstTimeSeriesMetadata(); + checked = true; } - return firstTimeSeriesMetadata != null; + if (checked && firstTimeSeriesMetadata == null) { + return Optional.empty(); + } + return Optional.of(firstTimeSeriesMetadata != null); } private boolean currentFileOverlapped() { @@ -299,11 +308,16 @@ public void skipCurrentFile() { * This method should be called after hasNextFile() until no next chunk, make sure that all * overlapped chunks are consumed. * + * @return Optional When Optional.empty() is returned, it means that the current + * hasNextFile has not been fully executed. In order to avoid the execution time of this + * method exceeding the allocated time slice, it is return early in this way. For the + * upper-level method, when encountering Optional.empty(), it needs to return directly to the + * checkpoint method who checks the operator execution time slice. * @throws IllegalStateException illegal state */ - public boolean hasNextChunk() throws IOException { + public Optional hasNextChunk() throws IOException { if (!paginationController.hasCurLimit()) { - return false; + return Optional.of(false); } if (!unSeqPageReaders.isEmpty() @@ -319,18 +333,28 @@ public boolean hasNextChunk() throws IOException { } if (firstChunkMetadata != null) { - return true; + return Optional.of(true); // hasNextFile() has not been invoked } else if (firstTimeSeriesMetadata == null && cachedChunkMetadata.isEmpty()) { - return false; + return Optional.of(false); } - while (firstChunkMetadata == null && (!cachedChunkMetadata.isEmpty() || hasNextFile())) { + Optional hasNextFileReturnValue = null; + while (firstChunkMetadata == null) { + if (cachedChunkMetadata.isEmpty()) { + if (hasNextFileReturnValue != null) { + return Optional.empty(); + } + hasNextFileReturnValue = hasNextFile(); + if (!hasNextFileReturnValue.isPresent() || !hasNextFileReturnValue.get()) { + return hasNextFileReturnValue; + } + } initFirstChunkMetadata(); // filter chunk based on push-down conditions filterFirstChunkMetadata(); } - return firstChunkMetadata != null; + return Optional.of(firstChunkMetadata != null); } private void filterFirstChunkMetadata() { @@ -1072,15 +1096,21 @@ private void tryToUnpackAllOverlappedFilesToTimeSeriesMetadata() throws IOExcept /* * Fill sequence TimeSeriesMetadata List until it is not empty */ - while (seqTimeSeriesMetadata.isEmpty() && orderUtils.hasNextSeqResource()) { - unpackSeqTsFileResource(); + if (seqTimeSeriesMetadata.isEmpty() && orderUtils.hasNextSeqResource()) { + // Avoid exceeding the time slice when a series cannot be found + if (!unpackSeqTsFileResource().isPresent()) { + return; + } } /* * Fill unSequence TimeSeriesMetadata Priority Queue until it is not empty */ - while (unSeqTimeSeriesMetadata.isEmpty() && orderUtils.hasNextUnseqResource()) { - unpackUnseqTsFileResource(); + if (unSeqTimeSeriesMetadata.isEmpty() && orderUtils.hasNextUnseqResource()) { + // Avoid exceeding the time slice when a series cannot be found + if (!unpackUnseqTsFileResource().isPresent()) { + return; + } } /* @@ -1187,13 +1217,16 @@ private Optional unpackSeqTsFileResource() throws IOExcepti } } - private void unpackUnseqTsFileResource() throws IOException { + private Optional unpackUnseqTsFileResource() throws IOException { ITimeSeriesMetadata timeseriesMetadata = loadTimeSeriesMetadata(orderUtils.getNextUnseqFileResource(true), false); // skip if data type is mismatched which may be caused by delete if (timeseriesMetadata != null && timeseriesMetadata.typeMatch(getTsDataTypeList())) { timeseriesMetadata.setSeq(false); unSeqTimeSeriesMetadata.add(timeseriesMetadata); + return Optional.of(timeseriesMetadata); + } else { + return Optional.empty(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java index dd96578ab92e..a59ec643b071 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractAggTableScanOperator.java @@ -193,48 +193,63 @@ protected void constructAlignedSeriesScanUtil() { } /** Return true if we have the result of this timeRange. */ - protected boolean calculateAggregationResultForCurrentTimeRange() { + protected Optional calculateAggregationResultForCurrentTimeRange() { try { if (calcFromCachedData()) { updateResultTsBlock(); checkIfAllAggregatorHasFinalResult(); - return true; + return Optional.of(true); } if (readAndCalcFromPage()) { updateResultTsBlock(); checkIfAllAggregatorHasFinalResult(); - return true; + return Optional.of(true); } // only when all the page data has been consumed, we need to read the chunk data if (!seriesScanUtil.hasNextPage() && readAndCalcFromChunk()) { updateResultTsBlock(); checkIfAllAggregatorHasFinalResult(); - return true; + return Optional.of(true); } // only when all the page and chunk data has been consumed, we need to read the file data - if (!seriesScanUtil.hasNextPage() - && !seriesScanUtil.hasNextChunk() - && readAndCalcFromFile()) { - updateResultTsBlock(); - checkIfAllAggregatorHasFinalResult(); - return true; + if (!seriesScanUtil.hasNextPage()) { + Optional b = seriesScanUtil.hasNextChunk(); + if (!b.isPresent()) { + return b; + } + if (!b.get() && readAndCalcFromFile()) { + updateResultTsBlock(); + checkIfAllAggregatorHasFinalResult(); + return Optional.of(true); + } } // If the TimeRange is (Long.MIN_VALUE, Long.MAX_VALUE), for Aggregators like countAggregator, // we have to consume all the data before we finish the aggregation calculation. - if (seriesScanUtil.hasNextPage() - || seriesScanUtil.hasNextChunk() - || seriesScanUtil.hasNextFile()) { - return false; - } else { - // all data of current device has been consumed - updateResultTsBlock(); - timeIterator.resetCurTimeRange(); - nextDevice(); + if (seriesScanUtil.hasNextPage()) { + return Optional.of(false); + } + Optional b = seriesScanUtil.hasNextChunk(); + if (!b.isPresent()) { + return b; + } + if (b.get()) { + return Optional.of(false); + } + b = seriesScanUtil.hasNextFile(); + if (!b.isPresent()) { + return b; } + if (b.get()) { + return Optional.of(false); + } + // all data of current device has been consumed + updateResultTsBlock(); + timeIterator.resetCurTimeRange(); + nextDevice(); if (currentDeviceIndex < deviceCount) { // construct AlignedSeriesScanUtil for next device @@ -246,9 +261,9 @@ && readAndCalcFromFile()) { if (currentDeviceIndex >= deviceCount) { // all devices have been consumed timeIterator.setFinished(); - return true; + return Optional.of(true); } else { - return false; + return Optional.of(false); } } catch (IOException e) { throw new RuntimeException("Error while scanning the file", e); @@ -452,7 +467,14 @@ private Statistics getStatistics(Statistics timeStatistics, Binary columnName) { public boolean readAndCalcFromFile() throws IOException { // start stopwatch long start = System.nanoTime(); - while (System.nanoTime() - start < leftRuntimeOfOneNextCall && seriesScanUtil.hasNextFile()) { + while (System.nanoTime() - start < leftRuntimeOfOneNextCall) { + Optional b = seriesScanUtil.hasNextFile(); + if (!b.isPresent()) { + continue; + } + if (!b.get()) { + break; + } if (canUseStatistics && seriesScanUtil.canUseCurrentFileStatistics()) { Statistics fileTimeStatistics = seriesScanUtil.currentFileTimeStatistics(); @@ -498,7 +520,14 @@ public boolean readAndCalcFromFile() throws IOException { protected boolean readAndCalcFromChunk() throws IOException { // start stopwatch long start = System.nanoTime(); - while (System.nanoTime() - start < leftRuntimeOfOneNextCall && seriesScanUtil.hasNextChunk()) { + while (System.nanoTime() - start < leftRuntimeOfOneNextCall) { + Optional b = seriesScanUtil.hasNextChunk(); + if (!b.isPresent()) { + continue; + } + if (!b.get()) { + break; + } if (canUseStatistics && seriesScanUtil.canUseCurrentChunkStatistics()) { Statistics chunkTimeStatistics = seriesScanUtil.currentChunkTimeStatistics(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java index ba7e1a226fef..56eddc11533e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractDefaultAggTableScanOperator.java @@ -27,6 +27,7 @@ import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; import org.apache.tsfile.utils.RamUsageEstimator; +import java.util.Optional; import java.util.concurrent.TimeUnit; import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.AbstractTableScanOperator.TIME_COLUMN_TEMPLATE; @@ -77,7 +78,8 @@ public TsBlock next() throws Exception { // calculate aggregation result on current time window // return true if current time window is calc finished - if (calculateAggregationResultForCurrentTimeRange()) { + Optional b = calculateAggregationResultForCurrentTimeRange(); + if (b.isPresent() && b.get()) { timeIterator.resetCurTimeRange(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java index 96de6f92e107..151dd6f3f259 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/AbstractTableScanOperator.java @@ -142,11 +142,19 @@ public TsBlock next() throws Exception { * 2. consume chunk data secondly * 3. consume next file finally */ - if (!readPageData() && !readChunkData() && !readFileData()) { - currentDeviceNoMoreData = true; - break; + if (readPageData()) { + continue; } - + Optional b = readChunkData(); + if (!b.isPresent() || b.get()) { + continue; + } + b = readFileData(); + if (!b.isPresent() || b.get()) { + continue; + } + currentDeviceNoMoreData = true; + break; } while (System.nanoTime() - start < maxRuntime && !measurementDataBuilder.isFull() && measurementDataBlock == null); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/LastQueryAggTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/LastQueryAggTableScanOperator.java index 9b34de4b7ddc..df487f06c602 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/LastQueryAggTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/LastQueryAggTableScanOperator.java @@ -44,6 +44,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.OptionalLong; import java.util.concurrent.TimeUnit; @@ -147,7 +148,8 @@ private void processCurrentDevice() { return; } - if (calculateAggregationResultForCurrentTimeRange()) { + Optional b = calculateAggregationResultForCurrentTimeRange(); + if (b.isPresent() && b.get()) { timeIterator.resetCurTimeRange(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/reader/SeriesDataBlockReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/reader/SeriesDataBlockReader.java index 4df98ddb9bb4..a49b97c8f522 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/reader/SeriesDataBlockReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/reader/SeriesDataBlockReader.java @@ -37,6 +37,7 @@ import java.io.IOException; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; public class SeriesDataBlockReader implements IDataBlockReader { @@ -134,7 +135,14 @@ public boolean hasNextBatch() throws IOException { /* * consume next file finally */ - while (seriesScanUtil.hasNextFile()) { + while (true) { + Optional b = seriesScanUtil.hasNextFile(); + if (!b.isPresent()) { + continue; + } + if (!b.get()) { + break; + } if (readChunkData()) { hasCachedBatchData = true; return true; @@ -158,7 +166,15 @@ public void close() throws IOException { } private boolean readChunkData() throws IOException { - while (seriesScanUtil.hasNextChunk()) { + while (true) { + Optional b = seriesScanUtil.hasNextChunk(); + if (!b.isPresent()) { + // This reader is used for compaction, just keep traversing + continue; + } + if (!b.get()) { + break; + } if (readPageData()) { return true; } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSeriesScanLimitOffsetPushDownTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSeriesScanLimitOffsetPushDownTest.java index 8370634070a7..8dddda60c1ed 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSeriesScanLimitOffsetPushDownTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSeriesScanLimitOffsetPushDownTest.java @@ -67,8 +67,8 @@ private AlignedSeriesScanUtil getAlignedSeriesScanUtil(long limit, long offset) public void testSkipFile() throws IllegalPathException, IOException { AlignedSeriesScanUtil seriesScanUtil = getAlignedSeriesScanUtil(5, 10); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); @@ -79,26 +79,27 @@ public void testSkipFile() throws IllegalPathException, IOException { } Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } @Test public void testCannotSkipFile() throws IllegalPathException, IOException { AlignedSeriesScanUtil seriesScanUtil = getAlignedSeriesScanUtil(5, 20); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); @@ -113,18 +114,18 @@ public void testCannotSkipFile() throws IllegalPathException, IOException { public void testSkipChunk() throws IllegalPathException, IOException { AlignedSeriesScanUtil seriesScanUtil = getAlignedSeriesScanUtil(5, 30); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); @@ -139,28 +140,28 @@ public void testSkipChunk() throws IllegalPathException, IOException { public void testCannotSkipChunk() throws IllegalPathException, IOException { AlignedSeriesScanUtil seriesScanUtil = getAlignedSeriesScanUtil(5, 40); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); @@ -175,28 +176,28 @@ public void testCannotSkipChunk() throws IllegalPathException, IOException { public void testSkipPage() throws IllegalPathException, IOException { AlignedSeriesScanUtil seriesScanUtil = getAlignedSeriesScanUtil(5, 50); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); @@ -211,35 +212,35 @@ public void testSkipPage() throws IllegalPathException, IOException { public void testCannotSkipPage() throws IllegalPathException, IOException { AlignedSeriesScanUtil seriesScanUtil = getAlignedSeriesScanUtil(5, 60); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); @@ -254,35 +255,35 @@ public void testCannotSkipPage() throws IllegalPathException, IOException { public void testSkipPoint() throws IllegalPathException, IOException { AlignedSeriesScanUtil seriesScanUtil = getAlignedSeriesScanUtil(10, 75); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSeriesScanPredicatePushDownTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSeriesScanPredicatePushDownTest.java index 9a735ab7bec6..8c1eab17dc5e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSeriesScanPredicatePushDownTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSeriesScanPredicatePushDownTest.java @@ -68,16 +68,17 @@ private AlignedSeriesScanUtil getAlignedSeriesScanUtil( } @Test - @SuppressWarnings("squid:S5961") // Suppress "Test methods should not contain too many assertions" + @SuppressWarnings( + "squid:S5961") // Suppress "Test methods should not contain too manya assertions" public void testNoFilter() throws IllegalPathException, IOException { AlignedSeriesScanUtil seriesScanUtil = getAlignedSeriesScanUtil(null, null); // File 1 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertTrue(seriesScanUtil.canUseCurrentFileStatistics()); // File 1 - Chunk 1 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 1 - Chunk 1 - Page 1 @@ -86,14 +87,14 @@ public void testNoFilter() throws IllegalPathException, IOException { TsBlock tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(10, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); // File 2 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertTrue(seriesScanUtil.canUseCurrentFileStatistics()); // File 2 - Chunk 1 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 2 - Chunk 1 - Page 1 @@ -102,14 +103,14 @@ public void testNoFilter() throws IllegalPathException, IOException { tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(10, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); // File 3 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertTrue(seriesScanUtil.canUseCurrentFileStatistics()); // File 3 - Chunk 1 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 3 - Chunk 1 - Page 1 @@ -120,7 +121,7 @@ public void testNoFilter() throws IllegalPathException, IOException { Assert.assertFalse(seriesScanUtil.hasNextPage()); // File 3 - Chunk 2 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 3 - Chunk 2 - Page 1 @@ -129,14 +130,14 @@ public void testNoFilter() throws IllegalPathException, IOException { tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(10, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); // File 4 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertFalse(seriesScanUtil.canUseCurrentFileStatistics()); // File 4 - Chunk 1 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 4 - Chunk 1 - Page 1 (chunk actually) @@ -147,7 +148,7 @@ public void testNoFilter() throws IllegalPathException, IOException { Assert.assertFalse(seriesScanUtil.hasNextPage()); // File 4 - Chunk 1 - Page 2 (chunk actually) - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); Assert.assertTrue(seriesScanUtil.canUseCurrentPageStatistics()); tsBlock = seriesScanUtil.nextPage(); @@ -155,7 +156,7 @@ public void testNoFilter() throws IllegalPathException, IOException { Assert.assertFalse(seriesScanUtil.hasNextPage()); // File 4 - Chunk 1 - Page 3 (chunk actually) - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); Assert.assertTrue(seriesScanUtil.canUseCurrentPageStatistics()); tsBlock = seriesScanUtil.nextPage(); @@ -163,7 +164,7 @@ public void testNoFilter() throws IllegalPathException, IOException { Assert.assertFalse(seriesScanUtil.hasNextPage()); // (File 4 - Chunk 2) merge (File 5 - Chunk 1) - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertFalse(seriesScanUtil.canUseCurrentChunkStatistics()); // (File 4 - Chunk 2 - Page 1) merge (File 5 - Chunk 1 - Page 1) @@ -178,8 +179,8 @@ public void testNoFilter() throws IllegalPathException, IOException { tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(10, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } @Test @@ -194,11 +195,11 @@ public void testSkipWithFilter() throws IllegalPathException, IOException { // File 1 skipped // File 2 skipped // File 3 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertFalse(seriesScanUtil.canUseCurrentFileStatistics()); // File 3 - Chunk 1 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 3 - Chunk 1 - Page 1 @@ -213,7 +214,7 @@ public void testSkipWithFilter() throws IllegalPathException, IOException { // File 3 - Chunk 2 skipped // File 4 - Chunk 1 skipped // (File 4 - Chunk 2) merge (File 5 - Chunk 1) - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertFalse(seriesScanUtil.canUseCurrentChunkStatistics()); Assert.assertTrue(seriesScanUtil.hasNextPage()); Assert.assertFalse(seriesScanUtil.canUseCurrentPageStatistics()); @@ -226,7 +227,7 @@ public void testSkipWithFilter() throws IllegalPathException, IOException { Assert.assertTrue(tsBlock == null || tsBlock.isEmpty()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSingleColumnSeriesScanLimitOffsetPushDownTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSingleColumnSeriesScanLimitOffsetPushDownTest.java index 821281a77beb..44e7b87d9035 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSingleColumnSeriesScanLimitOffsetPushDownTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/AlignedSingleColumnSeriesScanLimitOffsetPushDownTest.java @@ -70,18 +70,18 @@ public void testSkipFile() throws IllegalPathException, IOException { // File 1 skipped // File 2 - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); @@ -98,21 +98,21 @@ public void testSkipChunk() throws IllegalPathException, IOException { // File 1 skipped (10 points) // File 2 skipped (6 points) - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); // File 3 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); // File 3 Chunk 1 skipped (10 points) // File 3 Chunk 2 (6 points should skip 4 points) - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); @@ -124,11 +124,11 @@ public void testSkipChunk() throws IllegalPathException, IOException { // remaining 3 points selected Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); // File 3 - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(3, tsBlock.getPositionCount()); @@ -138,8 +138,8 @@ public void testSkipChunk() throws IllegalPathException, IOException { } Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } @Test @@ -148,36 +148,36 @@ public void testSkipPage() throws IllegalPathException, IOException { // File 1 skipped (10 points) // File 2 skipped (6 points) - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); // File 3 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); // File 3 - Chunk 1 skipped (10 points) // File 3 - Chunk 2 skipped (6 points) - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(0, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); // File 4 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); // File 4 - Chunk 1 - Page 1 skipped (10 points) // File 4 - Chunk 1 - Page 2 (6 points should skip 3 points) - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(3, tsBlock.getPositionCount()); @@ -188,7 +188,7 @@ public void testSkipPage() throws IllegalPathException, IOException { Assert.assertFalse(seriesScanUtil.hasNextPage()); // File 4 - Chunk 1 - Page 2 (remaining 2 points) - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesReaderTestUtil.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesReaderTestUtil.java index 66e96378f091..c6c3bcb698aa 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesReaderTestUtil.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesReaderTestUtil.java @@ -46,6 +46,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import static org.apache.iotdb.commons.conf.IoTDBConstant.PATH_SEPARATOR; @@ -216,4 +217,20 @@ private static void removeFiles( FileReaderManager.getInstance().closeAndRemoveAllOpenedReaders(); } + + static void assertWithHasNext(SeriesScanHasNextSupplier supplier, boolean value) + throws IOException { + while (true) { + Optional b = supplier.get(); + if (!b.isPresent()) { + continue; + } + Assert.assertEquals(b.get(), value); + break; + } + } + + interface SeriesScanHasNextSupplier { + Optional get() throws IOException; + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesScanLimitOffsetPushDownTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesScanLimitOffsetPushDownTest.java index 7eb8402e9301..c44d728640f7 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesScanLimitOffsetPushDownTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesScanLimitOffsetPushDownTest.java @@ -60,8 +60,8 @@ private SeriesScanUtil getSeriesScanUtil(long limit, long offset, Ordering scanO public void testSkipFile() throws IllegalPathException, IOException { SeriesScanUtil seriesScanUtil = getSeriesScanUtil(5, 10, Ordering.ASC); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); @@ -72,16 +72,16 @@ public void testSkipFile() throws IllegalPathException, IOException { } Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } @Test public void testSkipChunk() throws IllegalPathException, IOException { SeriesScanUtil seriesScanUtil = getSeriesScanUtil(5, 20, Ordering.ASC); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); @@ -92,16 +92,16 @@ public void testSkipChunk() throws IllegalPathException, IOException { } Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } @Test public void testSkipPage() throws IllegalPathException, IOException { SeriesScanUtil seriesScanUtil = getSeriesScanUtil(5, 30, Ordering.ASC); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); @@ -112,16 +112,16 @@ public void testSkipPage() throws IllegalPathException, IOException { } Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } @Test public void testSkipPoint1() throws IllegalPathException, IOException { SeriesScanUtil seriesScanUtil = getSeriesScanUtil(10, 45, Ordering.ASC); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); @@ -135,7 +135,7 @@ public void testSkipPoint1() throws IllegalPathException, IOException { Assert.assertEquals(expectedTime++, tsBlock.getTimeByIndex(i)); } - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(5, tsBlock.getPositionCount()); @@ -144,16 +144,16 @@ public void testSkipPoint1() throws IllegalPathException, IOException { } Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } @Test public void testSkipPoint2() throws IllegalPathException, IOException { SeriesScanUtil seriesScanUtil = getSeriesScanUtil(10, 55, Ordering.ASC); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); @@ -171,16 +171,16 @@ public void testSkipPoint2() throws IllegalPathException, IOException { } Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } @Test public void testSkipPointDesc1() throws IllegalPathException, IOException { SeriesScanUtil seriesScanUtil = getSeriesScanUtil(10, 5, Ordering.DESC); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); @@ -201,16 +201,16 @@ public void testSkipPointDesc1() throws IllegalPathException, IOException { } Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } @Test public void testSkipPointDesc2() throws IllegalPathException, IOException { SeriesScanUtil seriesScanUtil = getSeriesScanUtil(10, 25, Ordering.DESC); - Assert.assertTrue(seriesScanUtil.hasNextFile()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); TsBlock tsBlock = seriesScanUtil.nextPage(); @@ -223,7 +223,7 @@ public void testSkipPointDesc2() throws IllegalPathException, IOException { Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.hasNextPage()); tsBlock = seriesScanUtil.nextPage(); @@ -244,7 +244,7 @@ public void testSkipPointDesc2() throws IllegalPathException, IOException { } Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesScanPredicatePushDownTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesScanPredicatePushDownTest.java index 4b026873d8cb..4a546d787909 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesScanPredicatePushDownTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/series/SeriesScanPredicatePushDownTest.java @@ -67,11 +67,11 @@ public void testNoFilter() throws IllegalPathException, IOException { SeriesScanUtil seriesScanUtil = getSeriesScanUtil(null, null); // File 1 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertTrue(seriesScanUtil.canUseCurrentFileStatistics()); // File 1 - Chunk 1 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 1 - Chunk 1 - Page 1 @@ -80,14 +80,14 @@ public void testNoFilter() throws IllegalPathException, IOException { TsBlock tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(10, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); // File 2 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertTrue(seriesScanUtil.canUseCurrentFileStatistics()); // File 2 - Chunk 1 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 2 - Chunk 1 - Page 1 @@ -98,7 +98,7 @@ public void testNoFilter() throws IllegalPathException, IOException { Assert.assertFalse(seriesScanUtil.hasNextPage()); // File 2 - Chunk 2 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 2 - Chunk 2 - Page 1 @@ -107,14 +107,14 @@ public void testNoFilter() throws IllegalPathException, IOException { tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(10, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); // File 3 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertFalse(seriesScanUtil.canUseCurrentFileStatistics()); // File 3 - Chunk 1 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 3 - Chunk 1 - Page 1 @@ -131,7 +131,7 @@ public void testNoFilter() throws IllegalPathException, IOException { Assert.assertFalse(seriesScanUtil.hasNextPage()); // (File 3 - Chunk 2) merge (File 4 - Chunk 1) - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertFalse(seriesScanUtil.canUseCurrentChunkStatistics()); // (File 3 - Chunk 2 - Page 1) merge (File 4 - Chunk 1 - Page 1) @@ -146,8 +146,8 @@ public void testNoFilter() throws IllegalPathException, IOException { tsBlock = seriesScanUtil.nextPage(); Assert.assertEquals(10, tsBlock.getPositionCount()); Assert.assertFalse(seriesScanUtil.hasNextPage()); - Assert.assertFalse(seriesScanUtil.hasNextChunk()); - Assert.assertFalse(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, false); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, false); } @Test @@ -167,11 +167,11 @@ public void testSkipFileByPushDownFilter() throws IllegalPathException, IOExcept private void checkFile1Skipped(SeriesScanUtil seriesScanUtil) throws IOException { // File 1 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertTrue(seriesScanUtil.canUseCurrentFileStatistics()); // File 1 - Chunk 1 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 1 - Chunk 1 - Page 1 @@ -200,12 +200,12 @@ public void testSkipChunkByPushDownFilter() throws IllegalPathException, IOExcep private void checkFile2Chunk1Skipped(SeriesScanUtil seriesScanUtil) throws IOException { // File 1 skipped // File 2 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertFalse(seriesScanUtil.canUseCurrentFileStatistics()); // File 2 - Chunk 1 skipped // File 2 - Chunk 2 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertTrue(seriesScanUtil.canUseCurrentChunkStatistics()); // File 2 - Chunk 2 - Page 1 @@ -256,11 +256,11 @@ private void checkFile1AndFile2Skipped(SeriesScanUtil seriesScanUtil) throws IOE // File 1 skipped // File 2 skipped // File 3 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertFalse(seriesScanUtil.canUseCurrentFileStatistics()); // File 3 - Chunk 1 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertFalse(seriesScanUtil.canUseCurrentChunkStatistics()); } @@ -285,12 +285,12 @@ private void checkFile1AndFile2AndFile3Chunk1Skipped(SeriesScanUtil seriesScanUt // File 1 skipped // File 2 skipped // File 3 - Assert.assertTrue(seriesScanUtil.hasNextFile()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextFile, true); Assert.assertFalse(seriesScanUtil.canUseCurrentFileStatistics()); // File 3 - Chunk 1 skipped // File 3 - Chunk 2 - Assert.assertTrue(seriesScanUtil.hasNextChunk()); + SeriesReaderTestUtil.assertWithHasNext(seriesScanUtil::hasNextChunk, true); Assert.assertFalse(seriesScanUtil.canUseCurrentChunkStatistics()); }