diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/query/IoTDBFilteredRowsIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/query/IoTDBFilteredRowsIT.java new file mode 100644 index 0000000000000..334d3dee61bea --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/query/IoTDBFilteredRowsIT.java @@ -0,0 +1,364 @@ +/* + * 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.iotdb.db.it.query; + +import org.apache.iotdb.isession.ISession; +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.LocalStandaloneIT; +import org.apache.iotdb.rpc.IoTDBConnectionException; +import org.apache.iotdb.rpc.StatementExecutionException; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.IMeasurementSchema; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; + +@RunWith(IoTDBTestRunner.class) +@Category({LocalStandaloneIT.class}) +public class IoTDBFilteredRowsIT { + + private static final Logger log = LoggerFactory.getLogger(IoTDBFilteredRowsIT.class); + + private static final String DATABASE = "root.filter_info"; + private static final int MAX_NUMBER_OF_POINTS_IN_PAGE = 10; + private final List TARGET_KEYS = + Arrays.asList( + "timeSeriesIndexFilteredRows", + "chunkIndexFilteredRows", + "pageIndexFilteredRows", + "rowScanFilteredRows"); + + @BeforeClass + public static void setUp() throws Exception { + EnvFactory.getEnv() + .getConfig() + .getCommonConfig() + .setEnableSeqSpaceCompaction(false) + .setEnableUnseqSpaceCompaction(false) + .setEnableCrossSpaceCompaction(false) + .setMaxNumberOfPointsInPage(MAX_NUMBER_OF_POINTS_IN_PAGE); + EnvFactory.getEnv().initClusterEnvironment(); + } + + @AfterClass + public static void tearDown() throws Exception { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + + private static void prepareData( + ISession session, String device, boolean isFlush, boolean isAligned) + throws IoTDBConnectionException, StatementExecutionException { + + generateTimeRangeWithTimestamp(session, device, 1, 100, isFlush, isAligned); + generateTimeRangeWithTimestamp(session, device, 101, 200, isFlush, isAligned); + generateTimeRangeWithTimestamp(session, device, 201, 300, isFlush, isAligned); + generateTimeRangeWithTimestamp(session, device, 301, 400, isFlush, isAligned); + } + + private void verifyExplainMetrics( + Connection connection, + String condition, + long expectedTimeSeries, + long expectedChunk, + long expectedPage, + long expectedRowScan, + String device) + throws Exception { + + try (Statement statement = connection.createStatement()) { + String sql = "explain analyze verbose select * from " + device + " where " + condition; + log.info(sql); + ResultSet resultSet = statement.executeQuery(sql); + + StringBuilder stringBuilder = new StringBuilder(); + while (resultSet.next()) { + stringBuilder.append(resultSet.getString(1)).append(System.lineSeparator()); + } + + String[] allInfo = stringBuilder.toString().split(System.lineSeparator()); + List filteredRows = + Arrays.stream(allInfo) + .filter(line -> TARGET_KEYS.stream().anyMatch(line::contains)) + .map(line -> Long.parseLong(line.split(":")[1].trim())) + .collect(Collectors.toList()); + + assertEquals( + "timeSeriesIndexFilteredRows mismatch " + condition, + expectedTimeSeries, + (long) filteredRows.get(0)); + assertEquals( + "chunkIndexFilteredRows mismatch " + condition, + expectedChunk, + (long) filteredRows.get(1)); + assertEquals( + "pageIndexFilteredRows mismatch " + condition, expectedPage, (long) filteredRows.get(2)); + assertEquals( + "rowScanFilteredRows mismatch " + condition, expectedRowScan, (long) filteredRows.get(3)); + } + } + + private static void generateTimeRangeWithTimestamp( + ISession session, String device, long start, long end, boolean isFlush, boolean isAligned) + throws IoTDBConnectionException, StatementExecutionException { + + // if the TSDataType is TEXT, the related filter (where col_text = 'text_001') would n + List measurementSchemas = + Arrays.asList( + new MeasurementSchema("s_int32", TSDataType.INT32), + new MeasurementSchema("s_int64", TSDataType.INT64), + new MeasurementSchema("s_boolean", TSDataType.BOOLEAN), + new MeasurementSchema("s_float", TSDataType.FLOAT), + new MeasurementSchema("s_double", TSDataType.DOUBLE), + new MeasurementSchema("s_string", TSDataType.STRING)); + + Tablet tablet = new Tablet(device, measurementSchemas); + for (long currentTime = start; currentTime <= end; currentTime++) { + int rowIndex = tablet.getRowSize(); + if (rowIndex == tablet.getMaxRowNumber()) { + if (isAligned) { + session.insertAlignedTablet(tablet); + } else { + session.insertTablet(tablet); + } + tablet.reset(); + rowIndex = 0; + } + tablet.addTimestamp(rowIndex, currentTime); + tablet.addValue("s_int32", rowIndex, (int) currentTime); + tablet.addValue("s_int64", rowIndex, (long) currentTime); + tablet.addValue("s_boolean", rowIndex, currentTime % 2 == 0); + tablet.addValue("s_float", rowIndex, (float) currentTime + 0.1f); + tablet.addValue("s_double", rowIndex, (double) currentTime + 0.1d); + tablet.addValue("s_string", rowIndex, String.format("string_%03d", currentTime)); + } + if (tablet.getRowSize() > 0) { + if (isAligned) { + session.insertAlignedTablet(tablet); + } else { + session.insertTablet(tablet); + } + } + if (isFlush) { + session.executeNonQueryStatement("flush"); + } + } + + @Test + public void testUnAlignedReadDataFromDisk() throws Exception { + try (ISession session = EnvFactory.getEnv().getSessionConnection(); + Connection connection = EnvFactory.getEnv().getConnection()) { + String device_for_disk = DATABASE + ".device1"; + prepareData(session, device_for_disk, true, false); + + // 1. test INT32 + verifyExplainMetrics(connection, "s_int32 > 247", 200, 0, 40, 7, device_for_disk); + + // 2. test INT64 + verifyExplainMetrics(connection, "s_int64 > 123", 100, 0, 20, 3, device_for_disk); + + // 3. test FLOAT + verifyExplainMetrics(connection, "s_float > 315.5", 300, 0, 10, 5, device_for_disk); + + // 4. test DOUBLE + verifyExplainMetrics(connection, "s_double > 292.5", 200, 0, 90, 2, device_for_disk); + + // 5. test BOOLEAN + verifyExplainMetrics(connection, "s_boolean = true", 0, 0, 0, 200, device_for_disk); + + // 6. test STRING + verifyExplainMetrics(connection, "s_string = 'string_248'", 300, 0, 90, 9, device_for_disk); + + // 7. test TIME, the first tsfile (timeRange: 0-100), and second tsfile (timeRange : 100-200) + // would be filtered and could not be counted + // For non-aligned series, each measurement is filtered independently + // before being merged into a row, so filtered row counts are accumulated per measurement. + verifyExplainMetrics(connection, "time > 252", 0, 0, 50 * 6, 2 * 6, device_for_disk); + + // 8. AND condition + verifyExplainMetrics( + connection, "s_int32 > 105 and s_int64 <= 375", 100, 0, 20, 10, device_for_disk); + + // 9. AND condition, the 250th row will be filtered by rowScanFilteredRows + verifyExplainMetrics( + connection, "s_int32 > 150 AND s_string < 'string_250'", 200, 0, 100, 1, device_for_disk); + + // 10. or condition + verifyExplainMetrics( + connection, "time > 105 or s_double > 376.5", 100, 0, 0, 5, device_for_disk); + } + } + + @Test + public void testUnAlignedReadDataFromMemory() throws Exception { + try (ISession session = EnvFactory.getEnv().getSessionConnection(); + Connection connection = EnvFactory.getEnv().getConnection()) { + String device_for_memory = DATABASE + ".device2"; + prepareData(session, device_for_memory, false, false); + + // 1. test INT32 + verifyExplainMetrics(connection, "s_int32 > 247", 0, 0, 0, 247, device_for_memory); + + // 2. test INT64 + verifyExplainMetrics(connection, "s_int64 > 123", 0, 0, 0, 123, device_for_memory); + + // 3. test FLOAT + verifyExplainMetrics(connection, "s_float > 315.5", 0, 0, 0, 315, device_for_memory); + + // 4. test DOUBLE + verifyExplainMetrics(connection, "s_double > 292.5", 0, 0, 0, 292, device_for_memory); + + // 5. test BOOLEAN + verifyExplainMetrics(connection, "s_boolean = true", 0, 0, 0, 200, device_for_memory); + + // 6. test STRING + verifyExplainMetrics(connection, "s_string = 'string_248'", 0, 0, 0, 399, device_for_memory); + + // 7. test TIME, all the rows will be counted + verifyExplainMetrics(connection, "time > 252", 0, 0, 0, 252 * 6, device_for_memory); + + // 8. AND condition + verifyExplainMetrics( + connection, "s_int32 > 105 and s_int64 <= 375", 0, 0, 0, 130, device_for_memory); + + // 9. AND condition + verifyExplainMetrics( + connection, "s_int32 > 150 AND s_string < 'string_250'", 0, 0, 0, 301, device_for_memory); + + // 10. or condition + verifyExplainMetrics( + connection, "time > 105 or s_double > 376.5", 0, 0, 0, 105, device_for_memory); + } + } + + @Test + public void testReadAlignedDataFromDisk() throws Exception { + try (ISession session = EnvFactory.getEnv().getSessionConnection(); + Connection connection = EnvFactory.getEnv().getConnection()) { + String aligned_device_for_disk = DATABASE + ".aligned_device1"; + prepareData(session, aligned_device_for_disk, true, true); + + // 1. test INT32 + verifyExplainMetrics(connection, "s_int32 > 247", 200, 0, 40, 7, aligned_device_for_disk); + + // 2. test INT64 + verifyExplainMetrics(connection, "s_int64 > 123", 100, 0, 20, 3, aligned_device_for_disk); + + // 3. test FLOAT + verifyExplainMetrics(connection, "s_float > 315.5", 300, 0, 10, 5, aligned_device_for_disk); + + // 4. test DOUBLE + verifyExplainMetrics(connection, "s_double > 292.5", 200, 0, 90, 2, aligned_device_for_disk); + + // 5. test BOOLEAN + verifyExplainMetrics(connection, "s_boolean = true", 0, 0, 0, 200, aligned_device_for_disk); + + // 6. test STRING + verifyExplainMetrics( + connection, "s_string = 'string_248'", 300, 0, 90, 9, aligned_device_for_disk); + + // 7. test TIME + verifyExplainMetrics(connection, "time > 252", 0, 0, 50, 2, aligned_device_for_disk); + + // 8. AND condition + verifyExplainMetrics( + connection, "s_int32 > 105 and s_int64 <= 375", 100, 0, 20, 10, aligned_device_for_disk); + + // 9. AND condition + verifyExplainMetrics( + connection, + "s_int32 > 150 AND s_string < 'string_250'", + 200, + 0, + 100, + 1, + aligned_device_for_disk); + + // 10. or condition + verifyExplainMetrics( + connection, "time > 105 or s_double > 376.5", 100, 0, 0, 5, aligned_device_for_disk); + } + } + + @Test + public void testReadAlignedDataFromMemory() throws Exception { + try (ISession session = EnvFactory.getEnv().getSessionConnection(); + Connection connection = EnvFactory.getEnv().getConnection()) { + String aligned_device_for_memory = DATABASE + ".aligned_device2"; + prepareData(session, aligned_device_for_memory, false, true); + + // 1. test INT32 + verifyExplainMetrics(connection, "s_int32 > 247", 0, 0, 0, 247, aligned_device_for_memory); + + // 2. test INT64 + verifyExplainMetrics(connection, "s_int64 > 123", 0, 0, 0, 123, aligned_device_for_memory); + + // 3. test FLOAT + verifyExplainMetrics(connection, "s_float > 315.5", 0, 0, 0, 315, aligned_device_for_memory); + + // 4. test DOUBLE + verifyExplainMetrics(connection, "s_double > 292.5", 0, 0, 0, 292, aligned_device_for_memory); + + // 5. test BOOLEAN + verifyExplainMetrics(connection, "s_boolean = true", 0, 0, 0, 200, aligned_device_for_memory); + + // 6. test STRING + verifyExplainMetrics( + connection, "s_string = 'string_248'", 0, 0, 0, 399, aligned_device_for_memory); + + // 7. test TIME + verifyExplainMetrics(connection, "time > 252", 0, 0, 0, 252, aligned_device_for_memory); + + // 8. AND condition + verifyExplainMetrics( + connection, "s_int32 > 105 and s_int64 <= 375", 0, 0, 0, 130, aligned_device_for_memory); + + // 9. AND condition + verifyExplainMetrics( + connection, + "s_int32 > 150 AND s_string < 'string_250'", + 0, + 0, + 0, + 301, + aligned_device_for_memory); + + // 10. or condition + verifyExplainMetrics( + connection, "time > 105 or s_double > 376.5", 0, 0, 0, 105, aligned_device_for_memory); + } + } +} diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBTableFilteredRowsIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBTableFilteredRowsIT.java new file mode 100644 index 0000000000000..ee0990c394be2 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/query/recent/IoTDBTableFilteredRowsIT.java @@ -0,0 +1,272 @@ +/* + * 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.iotdb.relational.it.query.recent; + +import org.apache.iotdb.isession.ITableSession; +import org.apache.iotdb.isession.SessionDataSet; +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.TableLocalStandaloneIT; +import org.apache.iotdb.rpc.IoTDBConnectionException; +import org.apache.iotdb.rpc.StatementExecutionException; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.IMeasurementSchema; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; + +@RunWith(IoTDBTestRunner.class) +@Category({TableLocalStandaloneIT.class}) +public class IoTDBTableFilteredRowsIT { + + private static final String TABLE_DATABASE = "filter_info"; + private static final int MAX_NUMBER_OF_POINTS_IN_PAGE = 10; + private final List targetKeys = + Arrays.asList( + "timeSeriesIndexFilteredRows", + "chunkIndexFilteredRows", + "pageIndexFilteredRows", + "rowScanFilteredRows"); + + @BeforeClass + public static void setUp() throws Exception { + EnvFactory.getEnv() + .getConfig() + .getCommonConfig() + .setEnableSeqSpaceCompaction(false) + .setEnableUnseqSpaceCompaction(false) + .setEnableCrossSpaceCompaction(false) + .setMaxNumberOfPointsInPage(MAX_NUMBER_OF_POINTS_IN_PAGE); + EnvFactory.getEnv().initClusterEnvironment(); + try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { + session.executeNonQueryStatement("create database " + TABLE_DATABASE); + } + } + + @AfterClass + public static void tearDown() throws Exception { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + + private static void prepareData(ITableSession session, String device, boolean isFlush) + throws IoTDBConnectionException, StatementExecutionException { + + // for device1 + session.executeNonQueryStatement("use " + TABLE_DATABASE); + + generateTimeRangeWithTimestamp(session, device, 1, 100, isFlush); + + generateTimeRangeWithTimestamp(session, device, 101, 200, isFlush); + + generateTimeRangeWithTimestamp(session, device, 201, 300, isFlush); + + generateTimeRangeWithTimestamp(session, device, 301, 400, isFlush); + } + + private void verifyExplainMetrics( + ITableSession session, + String condition, + long expectedTimeSeries, + long expectedChunk, + long expectedPage, + long expectedRowScan, + String device) + throws IoTDBConnectionException, StatementExecutionException { + + SessionDataSet sessionDataSet = + session.executeQueryStatement( + "explain analyze verbose select * from " + device + " where " + condition); + SessionDataSet.DataIterator iterator = sessionDataSet.iterator(); + + StringBuilder stringBuilder = new StringBuilder(); + while (iterator.next()) { + stringBuilder.append(iterator.getString(1)).append(System.lineSeparator()); + } + + String[] allInfo = stringBuilder.toString().split(System.lineSeparator()); + List filteredRows = + Arrays.stream(allInfo) + .filter(line -> targetKeys.stream().anyMatch(line::contains)) + .map(line -> Long.parseLong(line.split(":")[1].trim())) + .collect(Collectors.toList()); + + assertEquals( + "timeSeriesIndexFilteredRows mismatch " + condition, + expectedTimeSeries, + (long) filteredRows.get(0)); + assertEquals( + "chunkIndexFilteredRows mismatch " + condition, expectedChunk, (long) filteredRows.get(1)); + assertEquals( + "pageIndexFilteredRows mismatch " + condition, expectedPage, (long) filteredRows.get(2)); + assertEquals( + "rowScanFilteredRows mismatch " + condition, expectedRowScan, (long) filteredRows.get(3)); + } + + @Test + public void testReadDataFromDisk() throws Exception { + try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { + session.executeNonQueryStatement("use " + TABLE_DATABASE); + String device_for_disk = "device1"; + prepareData(session, device_for_disk, true); + + // 1. test INT32 + verifyExplainMetrics(session, "col_int32 > 247", 200, 0, 40, 7, device_for_disk); + + // 2. test INT64 + verifyExplainMetrics(session, "col_int64 > 123", 100, 0, 20, 3, device_for_disk); + + // 3. test FLOAT + verifyExplainMetrics(session, "col_float > 315.5", 300, 0, 10, 5, device_for_disk); + + // 4. test DOUBLE + verifyExplainMetrics(session, "col_double > 292.5", 200, 0, 90, 2, device_for_disk); + + // 5. test BOOLEAN + verifyExplainMetrics(session, "col_boolean = true", 0, 0, 0, 200, device_for_disk); + + // 6. test STRING + verifyExplainMetrics(session, "col_string = 'string_248'", 300, 0, 90, 9, device_for_disk); + + // 7. test TIME, the first tsfile (timeRange: 0-100), and second tsfile (timeRange : 100-200) + // would be filtered and could not be counted + verifyExplainMetrics(session, "time > 252", 0, 0, 50, 2, device_for_disk); + + // 8. AND condition + verifyExplainMetrics( + session, "col_int32 > 105 and col_int64 <= 375", 100, 0, 20, 10, device_for_disk); + + // 9. AND condition, the 250th row will be filtered by rowScanFilteredRows + verifyExplainMetrics( + session, + "col_int32 > 150 AND col_string < 'string_250'", + 200, + 0, + 100, + 1, + device_for_disk); + + // 10. or condition + verifyExplainMetrics( + session, "time > 105 or col_double > 376.5", 100, 0, 0, 5, device_for_disk); + } + } + + @Test + public void testReadDataFromMemory() throws Exception { + try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { + session.executeNonQueryStatement("use " + TABLE_DATABASE); + String device_for_memory = "device2"; + prepareData(session, device_for_memory, false); + + // 1. test INT32 + verifyExplainMetrics(session, "col_int32 > 247", 0, 0, 0, 247, device_for_memory); + + // 2. test INT64 + verifyExplainMetrics(session, "col_int64 > 123", 0, 0, 0, 123, device_for_memory); + + // 3. test FLOAT + verifyExplainMetrics(session, "col_float > 315.5", 0, 0, 0, 315, device_for_memory); + + // 4. test DOUBLE + verifyExplainMetrics(session, "col_double > 292.5", 0, 0, 0, 292, device_for_memory); + + // 5. test BOOLEAN + verifyExplainMetrics(session, "col_boolean = true", 0, 0, 0, 200, device_for_memory); + + // 6. test STRING + verifyExplainMetrics(session, "col_string = 'string_248'", 0, 0, 0, 399, device_for_memory); + + // 7. test TIME, all the rows will be counted + verifyExplainMetrics(session, "time > 252", 0, 0, 0, 252, device_for_memory); + + // 8. AND condition + verifyExplainMetrics( + session, "col_int32 > 105 and col_int64 <= 375", 0, 0, 0, 130, device_for_memory); + + // 9. AND condition + verifyExplainMetrics( + session, + "col_int32 > 150 AND col_string < 'string_250'", + 0, + 0, + 0, + 301, + device_for_memory); + + // 10. or condition + verifyExplainMetrics( + session, "time > 105 or col_double > 376.5", 0, 0, 0, 105, device_for_memory); + } + } + + private static void generateTimeRangeWithTimestamp( + ITableSession session, String device, long start, long end, boolean isFlush) + throws IoTDBConnectionException, StatementExecutionException { + List measurementSchemas = + Arrays.asList( + new MeasurementSchema("col_int32", TSDataType.INT32), + new MeasurementSchema("col_int64", TSDataType.INT64), + new MeasurementSchema("col_boolean", TSDataType.BOOLEAN), + new MeasurementSchema("col_float", TSDataType.FLOAT), + new MeasurementSchema("col_double", TSDataType.DOUBLE), + new MeasurementSchema("col_string", TSDataType.STRING)); + + Tablet tablet = new Tablet(device, measurementSchemas); + for (long currentTime = start; currentTime <= end; currentTime++) { + int rowIndex = tablet.getRowSize(); + if (rowIndex == tablet.getMaxRowNumber()) { + session.insert(tablet); + tablet.reset(); + rowIndex = 0; + } + tablet.addTimestamp(rowIndex, currentTime); + + // INT32 + tablet.addValue(rowIndex, 0, (int) currentTime); + // INT64 + tablet.addValue(rowIndex, 1, (long) currentTime); + // BOOLEAN + tablet.addValue(rowIndex, 2, currentTime % 2 == 0); + // FLOAT + tablet.addValue(rowIndex, 3, (float) currentTime + 0.1f); + // DOUBLE + tablet.addValue(rowIndex, 4, (double) currentTime + 0.1d); + // STRING, pad with zeros (e.g: "string_001") to ensure consistent char lengths. + tablet.addValue(rowIndex, 5, String.format("string_%03d", currentTime)); + } + if (tablet.getRowSize() > 0) { + session.insert(tablet); + } + if (isFlush) { + session.executeNonQueryStatement("flush"); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/AlignedSinglePageWholeChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/AlignedSinglePageWholeChunkReader.java index 28e8a1deab4b8..757e7f8240b72 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/AlignedSinglePageWholeChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/AlignedSinglePageWholeChunkReader.java @@ -37,6 +37,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import java.util.function.LongConsumer; /** * The {@link AlignedSinglePageWholeChunkReader} is used to read a whole single page aligned chunk @@ -58,9 +59,10 @@ public class AlignedSinglePageWholeChunkReader extends AbstractChunkReader { // deleted intervals of all the sub sensors private final List> valueDeleteIntervalsList = new ArrayList<>(); - public AlignedSinglePageWholeChunkReader(Chunk timeChunk, List valueChunkList) + public AlignedSinglePageWholeChunkReader( + Chunk timeChunk, List valueChunkList, LongConsumer filteredRowsRecord) throws IOException { - super(Long.MIN_VALUE, null); + super(Long.MIN_VALUE, null, filteredRowsRecord); this.timeChunkHeader = timeChunk.getHeader(); this.timeChunkDataBuffer = timeChunk.getData(); this.encryptParam = timeChunk.getEncryptParam(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/SinglePageWholeChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/SinglePageWholeChunkReader.java index ed831644988ff..2b2743bad18f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/SinglePageWholeChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/SinglePageWholeChunkReader.java @@ -43,7 +43,7 @@ public class SinglePageWholeChunkReader extends AbstractChunkReader { private final EncryptParameter encryptParam; public SinglePageWholeChunkReader(Chunk chunk) throws IOException { - super(Long.MIN_VALUE, null); + super(Long.MIN_VALUE, null, null); this.chunkHeader = chunk.getHeader(); this.chunkDataBuffer = chunk.getData(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/TsFileInsertionEventScanParser.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/TsFileInsertionEventScanParser.java index 5dc6228f1d125..2bd22c8b0bc07 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/TsFileInsertionEventScanParser.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/parser/scan/TsFileInsertionEventScanParser.java @@ -709,7 +709,7 @@ private boolean recordAlignedChunk(final List valueChunkList, final byte chunkReader = currentIsMultiPage ? new AlignedChunkReader(timeChunk, valueChunkList, filter) - : new AlignedSinglePageWholeChunkReader(timeChunk, valueChunkList); + : new AlignedSinglePageWholeChunkReader(timeChunk, valueChunkList, null); currentIsAligned = true; lastMarker = marker; return true; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java index 3bd1f28e6ba6e..4eea2457b5072 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java @@ -104,7 +104,7 @@ public enum ExplainType { // - EXPLAIN: Show the logical and physical query plan without execution // - EXPLAIN_ANALYZE: Execute the query and collect detailed execution statistics private ExplainType explainType = ExplainType.NONE; - private boolean isVerbose = false; + private boolean verbose = false; private QueryPlanStatistics queryPlanStatistics = null; @@ -353,11 +353,11 @@ public boolean isExplain() { } public void setVerbose(boolean verbose) { - isVerbose = verbose; + this.verbose = verbose; } public boolean isVerbose() { - return isVerbose; + return verbose; } public long getAnalyzeCost() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FakedFragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FakedFragmentInstanceContext.java index bdebbc9f28964..138bba8035e4b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FakedFragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FakedFragmentInstanceContext.java @@ -35,7 +35,7 @@ public class FakedFragmentInstanceContext extends FragmentInstanceContext { public FakedFragmentInstanceContext(Filter timeFilter, DataRegion dataRegion) { - super(0, new FakedMemoryReservationManager(), timeFilter, dataRegion, false); + super(0, new FakedMemoryReservationManager(), timeFilter, dataRegion, false, false); } public QueryDataSource getSharedQueryDataSource(IFullPath sourcePath) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index bb16bf73c16f5..df9ac2d2f936f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -166,9 +166,10 @@ public static FragmentInstanceContext createFragmentInstanceContext( FragmentInstanceId id, FragmentInstanceStateMachine stateMachine, SessionInfo sessionInfo, - boolean debug) { + boolean debug, + boolean isVerbose) { FragmentInstanceContext instanceContext = - new FragmentInstanceContext(id, stateMachine, sessionInfo, debug); + new FragmentInstanceContext(id, stateMachine, sessionInfo, debug, isVerbose); instanceContext.initialize(); instanceContext.start(); return instanceContext; @@ -181,9 +182,11 @@ public static FragmentInstanceContext createFragmentInstanceContext( SessionInfo sessionInfo, IDataRegionForQuery dataRegion, Filter timeFilter, - boolean debug) { + boolean debug, + boolean isVerbose) { FragmentInstanceContext instanceContext = - new FragmentInstanceContext(id, stateMachine, sessionInfo, dataRegion, timeFilter, debug); + new FragmentInstanceContext( + id, stateMachine, sessionInfo, dataRegion, timeFilter, debug, isVerbose); instanceContext.initialize(); instanceContext.start(); return instanceContext; @@ -196,7 +199,8 @@ public static FragmentInstanceContext createFragmentInstanceContext( IDataRegionForQuery dataRegion, TimePredicate globalTimePredicate, Map dataNodeQueryContextMap, - boolean debug) { + boolean debug, + boolean isVerbose) { FragmentInstanceContext instanceContext = new FragmentInstanceContext( id, @@ -205,14 +209,15 @@ public static FragmentInstanceContext createFragmentInstanceContext( dataRegion, globalTimePredicate, dataNodeQueryContextMap, - debug); + debug, + isVerbose); instanceContext.initialize(); instanceContext.start(); return instanceContext; } public static FragmentInstanceContext createFragmentInstanceContextForCompaction(long queryId) { - return new FragmentInstanceContext(queryId, null, null, null, false); + return new FragmentInstanceContext(queryId, null, null, null, false, false); } public void setQueryDataSourceType(QueryDataSourceType queryDataSourceType) { @@ -227,6 +232,7 @@ public static FragmentInstanceContext createFragmentInstanceContext( id, stateMachine, new SessionInfo(1, new UserEntity(666, "test", "127.0.0.1"), ZoneId.systemDefault()), + false, false); instanceContext.initialize(); instanceContext.start(); @@ -244,6 +250,7 @@ public static FragmentInstanceContext createFragmentInstanceContext( stateMachine, new SessionInfo(1, new UserEntity(666, "test", "127.0.0.1"), ZoneId.systemDefault()), memoryReservationManager, + false, false); instanceContext.initialize(); instanceContext.start(); @@ -257,8 +264,9 @@ private FragmentInstanceContext( IDataRegionForQuery dataRegion, TimePredicate globalTimePredicate, Map dataNodeQueryContextMap, - boolean debug) { - super(debug); + boolean debug, + boolean verbose) { + super(debug, verbose); this.id = id; this.stateMachine = stateMachine; this.executionEndTime.set(END_TIME_INITIAL_VALUE); @@ -279,8 +287,9 @@ private FragmentInstanceContext( FragmentInstanceId id, FragmentInstanceStateMachine stateMachine, SessionInfo sessionInfo, - boolean debug) { - super(debug); + boolean debug, + boolean verbose) { + super(debug, verbose); this.id = id; this.stateMachine = stateMachine; this.executionEndTime.set(END_TIME_INITIAL_VALUE); @@ -296,8 +305,9 @@ private FragmentInstanceContext( FragmentInstanceStateMachine stateMachine, SessionInfo sessionInfo, MemoryReservationManager memoryReservationManager, - boolean debug) { - super(debug); + boolean debug, + boolean verbose) { + super(debug, verbose); this.id = id; this.stateMachine = stateMachine; this.executionEndTime.set(END_TIME_INITIAL_VALUE); @@ -313,8 +323,9 @@ private FragmentInstanceContext( SessionInfo sessionInfo, IDataRegionForQuery dataRegion, Filter globalTimeFilter, - boolean debug) { - super(debug); + boolean debug, + boolean verbose) { + super(debug, verbose); this.id = id; this.stateMachine = stateMachine; this.executionEndTime.set(END_TIME_INITIAL_VALUE); @@ -337,8 +348,9 @@ protected FragmentInstanceContext( MemoryReservationManager memoryReservationManager, Filter timeFilter, DataRegion dataRegion, - boolean debug) { - super(debug); + boolean debug, + boolean verbose) { + super(debug, verbose); this.queryId = queryId; this.id = null; this.stateMachine = null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java index 9d099859e37d4..4e2319b5a1872 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecution.java @@ -250,10 +250,21 @@ private void setOperatorStatistics( operatorStatistics.setNextCalledCount(operatorContext.getNextCalledCount()); operatorStatistics.setHasNextCalledCount(operatorContext.getHasNextCalledCount()); operatorStatistics.setOutputRows(operatorContext.getOutputRows()); - operatorStatistics.setSpecifiedInfo(operatorContext.getSpecifiedInfo()); + operatorStatistics.setSpecifiedInfo(convertSpecifiedInfo(operatorContext.getSpecifiedInfo())); operatorStatistics.setMemoryUsage(operatorContext.getEstimatedMemorySize()); } + private Map convertSpecifiedInfo(Map specifiedInfo) { + Map result = new HashMap<>(specifiedInfo.size()); + specifiedInfo.forEach( + (key, value) -> { + if (value != null) { + result.put(key, value.toString()); + } + }); + return result; + } + // Directly build statistics from FragmentInstanceExecution, which is still running. public TFetchFragmentInstanceStatisticsResp buildStatistics() { TFetchFragmentInstanceStatisticsResp statistics = new TFetchFragmentInstanceStatisticsResp(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceManager.java index 7b5fbe3757140..3f6812943d6da 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceManager.java @@ -160,7 +160,8 @@ public FragmentInstanceInfo execDataQueryFragmentInstance( dataRegion, instance.getGlobalTimePredicate(), dataNodeQueryContextMap, - instance.isDebug())); + instance.isDebug(), + instance.isVerbose())); try { List driverFactories = @@ -273,7 +274,8 @@ public FragmentInstanceInfo execSchemaQueryFragmentInstance( fragmentInstanceId, stateMachine, instance.getSessionInfo(), - instance.isDebug())); + instance.isDebug(), + instance.isVerbose())); try { List driverFactories = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java index f165db769e99f..5330903412b2e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java @@ -64,6 +64,7 @@ public class QueryContext { protected long queryId; private final boolean debug; + private boolean verbose; private long startTime; private long timeout; @@ -79,8 +80,9 @@ public class QueryContext { protected Set tables; - public QueryContext(boolean debug) { + public QueryContext(boolean debug, boolean verbose) { this.debug = debug; + this.verbose = verbose; } public QueryContext(long queryId, boolean debug) { @@ -219,6 +221,10 @@ public boolean isDebug() { return debug; } + public boolean isVerbose() { + return verbose; + } + public long getStartTime() { return startTime; } @@ -264,4 +270,9 @@ public void setIgnoreAllNullRows(boolean ignoreAllNullRows) { public void addTVListToSet(Map tvListMap) { tvListSet.addAll(tvListMap.keySet()); } + + public void addRowLevelFilteredCount(long count) { + throw new UnsupportedOperationException( + "the QueryContext does not support row level filtering"); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryStatistics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryStatistics.java index 5291f96dfb51a..e426487f4e21c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryStatistics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryStatistics.java @@ -95,6 +95,12 @@ public class QueryStatistics { // statistics for count of chunk with metadata errors private final AtomicLong chunkWithMetadataErrorsCount = new AtomicLong(0); + // statistics for filtered data + private final AtomicLong timeSeriesIndexFilteredRows = new AtomicLong(0); + private final AtomicLong chunkIndexFilteredRows = new AtomicLong(0); + private final AtomicLong pageIndexFilteredRows = new AtomicLong(0); + private final AtomicLong rowScanFilteredRows = new AtomicLong(0); + public AtomicLong getLoadTimeSeriesMetadataDiskSeqCount() { return loadTimeSeriesMetadataDiskSeqCount; } @@ -295,6 +301,39 @@ public AtomicLong getChunkWithMetadataErrorsCount() { return chunkWithMetadataErrorsCount; } + public AtomicLong getTimeSeriesIndexFilteredRows() { + return this.timeSeriesIndexFilteredRows; + } + + public void addFilteredRowsOfTimeSeriesLevel(long count) { + this.timeSeriesIndexFilteredRows.addAndGet(count); + } + + public AtomicLong getChunkIndexFilteredRows() { + return this.chunkIndexFilteredRows; + } + + public void addFilteredRowsOfChunkLevel(long count) { + this.chunkIndexFilteredRows.addAndGet(count); + } + + public AtomicLong getPageIndexFilteredRows() { + return this.pageIndexFilteredRows; + } + + public void addFilteredRowsOfPageLevel(long count) { + this.pageIndexFilteredRows.addAndGet(count); + } + + // rows level, only explain analyze verbose would lead to recording the filtered data in row level + public AtomicLong getRowScanFilteredRows() { + return this.rowScanFilteredRows; + } + + public void addFilteredRowsOfRowLevel(long count) { + this.rowScanFilteredRows.addAndGet(count); + } + public TQueryStatistics toThrift() { return new TQueryStatistics( loadTimeSeriesMetadataDiskSeqCount.get(), @@ -344,6 +383,10 @@ public TQueryStatistics toThrift() { loadChunkFromCacheCount.get(), loadChunkFromDiskCount.get(), loadChunkActualIOSize.get(), - chunkWithMetadataErrorsCount.get()); + chunkWithMetadataErrorsCount.get(), + timeSeriesIndexFilteredRows.get(), + chunkIndexFilteredRows.get(), + pageIndexFilteredRows.get(), + rowScanFilteredRows.get()); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorContext.java index beb25030400fb..f8b594ff4bada 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorContext.java @@ -31,7 +31,6 @@ import org.apache.tsfile.utils.Accountable; import org.apache.tsfile.utils.RamUsageEstimator; -import java.util.HashMap; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; @@ -64,7 +63,7 @@ public class OperatorContext implements Accountable { // SpecifiedInfo is used to record some custom information for the operator, // which will be shown in the result of EXPLAIN ANALYZE to analyze the query. - private Map specifiedInfo = null; + private final Map specifiedInfo = new ConcurrentHashMap<>(); private long output = 0; private long estimatedMemorySize; @@ -169,15 +168,11 @@ public long getOutputRows() { } public void recordSpecifiedInfo(String key, String value) { - if (specifiedInfo == null) { - // explain analyze operator fetching and current operator updating may be concurrently - specifiedInfo = new ConcurrentHashMap<>(); - } specifiedInfo.put(key, value); } - public Map getSpecifiedInfo() { - return specifiedInfo == null ? new HashMap<>() : specifiedInfo; + public Map getSpecifiedInfo() { + return specifiedInfo; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/sink/IdentitySinkOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/sink/IdentitySinkOperator.java index be68eda524445..071061c29a926 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/sink/IdentitySinkOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/sink/IdentitySinkOperator.java @@ -38,6 +38,9 @@ public class IdentitySinkOperator implements Operator { RamUsageEstimator.shallowSizeOfInstance(IdentitySinkOperator.class) + RamUsageEstimator.shallowSizeOfInstance(DownStreamChannelIndex.class); + public static final String DOWNSTREAM_PLAN_NODE_ID = "DownStreamPlanNodeId"; + public static final String DELIMITER_BETWEEN_ID = ", "; + private final OperatorContext operatorContext; private final List children; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ExchangeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ExchangeOperator.java index cbc720839fd92..96233f17e0859 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ExchangeOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ExchangeOperator.java @@ -30,11 +30,16 @@ import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.utils.RamUsageEstimator; +import java.util.concurrent.atomic.AtomicLong; + public class ExchangeOperator implements SourceOperator { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(ExchangeOperator.class); + private final AtomicLong receivedSizeInBytes = new AtomicLong(0); + public static final String SIZE_IN_BYTES = "size_in_bytes"; + private final OperatorContext operatorContext; private final ISourceHandle sourceHandle; @@ -53,6 +58,7 @@ public ExchangeOperator( this.operatorContext = operatorContext; this.sourceHandle = sourceHandle; this.sourceId = sourceId; + this.operatorContext.getSpecifiedInfo().put(SIZE_IN_BYTES, receivedSizeInBytes); } /** @@ -70,6 +76,7 @@ public ExchangeOperator( this.sourceHandle = sourceHandle; this.sourceId = sourceId; this.maxReturnSize = maxReturnSize; + this.operatorContext.getSpecifiedInfo().put(SIZE_IN_BYTES, receivedSizeInBytes); } @Override @@ -79,7 +86,11 @@ public OperatorContext getOperatorContext() { @Override public TsBlock next() throws Exception { - return sourceHandle.receive(); + TsBlock receiveBlock = sourceHandle.receive(); + if (receiveBlock != null) { + receivedSizeInBytes.addAndGet(receiveBlock.getSizeInBytes()); + } + return receiveBlock; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java index 3eff3b9a53fbc..254c061187a43 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java @@ -151,6 +151,12 @@ public static TimeseriesMetadata loadTimeSeriesMetadata( return null; } if (globalTimeFilter != null && globalTimeFilter.canSkip(timeSeriesMetadata)) { + // for unclosed tsfile, the timeSeriesMetadata.getStatistics().getCount() may be + // inaccurate + // maybe return one but actual count is much more than one + context + .getQueryStatistics() + .addFilteredRowsOfTimeSeriesLevel(timeSeriesMetadata.getStatistics().getCount()); return null; } } @@ -228,6 +234,11 @@ public static AbstractAlignedTimeSeriesMetadata loadAlignedTimeSeriesMetadata( return null; } if (globalTimeFilter != null && globalTimeFilter.canSkip(alignedTimeSeriesMetadata)) { + // record the timeSeries level filtered data + context + .getQueryStatistics() + .addFilteredRowsOfTimeSeriesLevel( + alignedTimeSeriesMetadata.getStatistics().getCount()); return null; } } 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 fe9cfa9c0f8b0..8412161e63e39 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 @@ -415,6 +415,11 @@ private void filterFirstChunkMetadata() { // globalTimeFilter.canSkip() must be FALSE Filter pushDownFilter = scanOptions.getPushDownFilter(); if (pushDownFilter != null && pushDownFilter.canSkip(firstChunkMetadata)) { + + // record the chunk level filtered data + this.context + .getQueryStatistics() + .addFilteredRowsOfChunkLevel(firstChunkMetadata.getStatistics().getCount()); skipCurrentChunk(); return; } @@ -832,13 +837,30 @@ public void skipCurrentPage() { firstPageReader = null; } + /** + * Logic of Filter Application and Predicate Splitting: + * + *

1. Predicate Splitting (determined during query planning): - AND connection (e.g., `where s1 + * > 300 and time < 100`): The predicate is split. `time < 100` goes to `globalTimeFilter`, and + * `s1 > 300` goes to `pushDownFilter`. - OR connection (e.g., `where s1 > 300 or time < 100`): + * The entire predicate is assigned to `pushDownFilter`. `globalTimeFilter` remains empty (accepts + * all). + * + *

2. Filter Application in nextPage(): - Case 1: hasCachedNextOverlappedPage (Overlapped data + * processed by MergeReader) The `cachedTsBlock` comes from `mergeReader`. The source PageReaders + * of the mergeReader have already applied `globalTimeFilter` during their initialization/loading + * phase. Therefore, `filterAndPaginateCachedBlock(cachedTsBlock)` ONLY applies `pushDownFilter` + * (and pagination). + * + *

- Case 2: Non-overlapped data (firstPageReader) The `firstPageReader` also has + * `globalTimeFilter` applied internally. We explicitly call + * `firstPageReader.addPushDownFilter(...)` to ensure the value filter is applied. + */ public TsBlock nextPage() throws IOException { if (hasCachedNextOverlappedPage) { hasCachedNextOverlappedPage = false; - TsBlock res = - applyPushDownFilterAndLimitOffset( - cachedTsBlock, scanOptions.getPushDownFilter(), paginationController); + TsBlock res = filterAndPaginateCachedBlock(cachedTsBlock); cachedTsBlock = null; // cached tsblock has handled by pagination controller & push down filter, return directly @@ -886,6 +908,25 @@ public TsBlock nextPage() throws IOException { } } + private TsBlock filterAndPaginateCachedBlock(TsBlock tsBlock) { + if (scanOptions.getPushDownFilter() == null) { + return paginationController.applyTsBlock(tsBlock); + } + if (this.context.isVerbose()) { + return TsBlockUtil.applyFilterAndLimitOffsetToTsBlock( + tsBlock, + new TsBlockBuilder(getTsDataTypeList()), + scanOptions.getPushDownFilter(), + paginationController, + this.context.getQueryStatistics()::addFilteredRowsOfRowLevel); + } + return TsBlockUtil.applyFilterAndLimitOffsetToTsBlock( + tsBlock, + new TsBlockBuilder(getTsDataTypeList()), + scanOptions.getPushDownFilter(), + paginationController); + } + private TsBlock getTransferedDataTypeTsBlock(TsBlock tsBlock) { Column[] valueColumns = tsBlock.getValueColumns(); int length = tsBlock.getValueColumnCount(); @@ -1349,15 +1390,7 @@ private TsBlock getTransferedDataTypeTsBlock(TsBlock tsBlock) { return tsBlock; } - private TsBlock applyPushDownFilterAndLimitOffset( - TsBlock tsBlock, Filter pushDownFilter, PaginationController paginationController) { - if (pushDownFilter == null) { - return paginationController.applyTsBlock(tsBlock); - } - return TsBlockUtil.applyFilterAndLimitOffsetToTsBlock( - tsBlock, new TsBlockBuilder(getTsDataTypeList()), pushDownFilter, paginationController); - } - + /** filter data in whole page level, and apply the offset at the same time */ private void filterFirstPageReader() { if (firstPageReader == null || firstPageReader.isModified()) { return; @@ -1368,6 +1401,10 @@ private void filterFirstPageReader() { // globalTimeFilter.canSkip() must be FALSE Filter pushDownFilter = scanOptions.getPushDownFilter(); if (pushDownFilter != null && pushDownFilter.canSkip(pageReader)) { + // record the page level filtered data + this.context + .getQueryStatistics() + .addFilteredRowsOfPageLevel(pageReader.getStatistics().getCount()); skipCurrentPage(); return; } @@ -1839,7 +1876,8 @@ private void filterFirstTimeSeriesMetadata() { if (firstTimeSeriesMetadata == null) { return; } - + // if the time range is overLapped, current file cannot be considered as truth, so all filters + // are invalid if (currentFileOverlapped() || firstTimeSeriesMetadata.isModified()) { return; } @@ -1847,6 +1885,11 @@ private void filterFirstTimeSeriesMetadata() { // globalTimeFilter.canSkip() must be FALSE Filter pushDownFilter = scanOptions.getPushDownFilter(); if (pushDownFilter != null && pushDownFilter.canSkip(firstTimeSeriesMetadata)) { + + // record the timeSeries level filtered data + this.context + .getQueryStatistics() + .addFilteredRowsOfTimeSeriesLevel(firstTimeSeriesMetadata.getStatistics().getCount()); skipCurrentFile(); return; } @@ -2009,7 +2052,12 @@ public IPageReader getPageReader() { public TsBlock getAllSatisfiedPageData(boolean ascending) throws IOException { long startTime = System.nanoTime(); try { - TsBlock tsBlock = data.getAllSatisfiedData(); + + TsBlock tsBlock = + this.context.isVerbose() + ? data.getAllSatisfiedData( + this.context.getQueryStatistics()::addFilteredRowsOfRowLevel) + : data.getAllSatisfiedData(); if (!ascending) { tsBlock.reverse(); } 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 fa6a7093b5864..60bfd87ba316a 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 @@ -57,7 +57,6 @@ public abstract class AbstractTableScanOperator extends AbstractSeriesScanOperat RamUsageEstimator.shallowSizeOfInstance(TableScanOperator.class); public static final String CURRENT_DEVICE_INDEX_STRING = "CurrentDeviceIndex"; - public static final LongColumn TIME_COLUMN_TEMPLATE = new LongColumn(1, Optional.empty(), new long[] {0}); 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 9b4c37995de63..fcbcc8c9ecc5f 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 @@ -136,7 +136,6 @@ public boolean hasNext() throws Exception { public TsBlock next() throws Exception { long maxRuntime = operatorContext.getMaxRunTime().roundTo(TimeUnit.NANOSECONDS); long start = System.nanoTime(); - if (retainedTsBlock != null) { return getResultFromRetainedTsBlock(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index cea84ba5794bb..c67b3bd527d8c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -440,7 +440,8 @@ public ExecutionResult executeForTableModel( ExplainType explainType, long timeOut, boolean userQuery, - boolean debug) { + boolean debug, + boolean isVerbose) { return execution( queryId, session, @@ -451,6 +452,7 @@ public ExecutionResult executeForTableModel( queryContext.setInnerTriggeredQuery(true); queryContext.setCteQueries(cteQueries); queryContext.setExplainType(explainType); + queryContext.setVerbose(isVerbose); return createQueryExecutionForTableModel( statement, sqlParser, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java index f0f5d419f38ca..5765842b56624 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java @@ -253,6 +253,7 @@ public Analysis visitExplainAnalyze( ExplainAnalyzeStatement explainAnalyzeStatement, MPPQueryContext context) { Analysis analysis = visitQuery(explainAnalyzeStatement.getQueryStatement(), context); context.setExplainType(ExplainType.EXPLAIN_ANALYZE); + context.setVerbose(explainAnalyzeStatement.isVerbose()); analysis.setRealStatement(explainAnalyzeStatement); analysis.setRespDatasetHeader( new DatasetHeader( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java index f69c5aec0c650..bf760be834688 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java @@ -44,6 +44,7 @@ import org.apache.iotdb.db.queryengine.execution.exchange.MPPDataExchangeManager; import org.apache.iotdb.db.queryengine.execution.exchange.MPPDataExchangeService; import org.apache.iotdb.db.queryengine.execution.exchange.sink.DownStreamChannelIndex; +import org.apache.iotdb.db.queryengine.execution.exchange.sink.DownStreamChannelLocation; import org.apache.iotdb.db.queryengine.execution.exchange.sink.ISinkChannel; import org.apache.iotdb.db.queryengine.execution.exchange.sink.ISinkHandle; import org.apache.iotdb.db.queryengine.execution.exchange.sink.LocalSinkChannel; @@ -315,6 +316,8 @@ import static org.apache.iotdb.db.queryengine.execution.operator.AggregationUtil.getOutputColumnSizePerLine; import static org.apache.iotdb.db.queryengine.execution.operator.AggregationUtil.initTimeRangeIterator; import static org.apache.iotdb.db.queryengine.execution.operator.process.join.merge.MergeSortComparator.getComparator; +import static org.apache.iotdb.db.queryengine.execution.operator.sink.IdentitySinkOperator.DELIMITER_BETWEEN_ID; +import static org.apache.iotdb.db.queryengine.execution.operator.sink.IdentitySinkOperator.DOWNSTREAM_PLAN_NODE_ID; import static org.apache.iotdb.db.queryengine.plan.analyze.PredicateUtils.convertPredicateToFilter; import static org.apache.iotdb.db.queryengine.plan.expression.leaf.TimestampOperand.TIMESTAMP_EXPRESSION_STRING; import static org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.AggregationDescriptor.getAggregationTypeByFuncName; @@ -2676,6 +2679,13 @@ public Operator visitIdentitySink(IdentitySinkNode node, LocalExecutionPlanConte context.getNextOperatorId(), node.getPlanNodeId(), IdentitySinkOperator.class.getSimpleName()); + String downStreamPlanNodeId = + node.getDownStreamChannelLocationList().stream() + .map(DownStreamChannelLocation::getRemotePlanNodeId) + .collect(Collectors.joining(DELIMITER_BETWEEN_ID)); + if (!downStreamPlanNodeId.isEmpty()) { + operatorContext.recordSpecifiedInfo(DOWNSTREAM_PLAN_NODE_ID, downStreamPlanNodeId); + } checkArgument( MPP_DATA_EXCHANGE_MANAGER != null, "MPP_DATA_EXCHANGE_MANAGER should not be null"); @@ -2707,6 +2717,14 @@ public Operator visitShuffleSink(ShuffleSinkNode node, LocalExecutionPlanContext node.getPlanNodeId(), ShuffleHelperOperator.class.getSimpleName()); + String downStreamPlanNodeIds = + node.getDownStreamChannelLocationList().stream() + .map(DownStreamChannelLocation::getRemotePlanNodeId) + .collect(Collectors.joining(DELIMITER_BETWEEN_ID)); + if (!downStreamPlanNodeIds.isEmpty()) { + operatorContext.recordSpecifiedInfo(DOWNSTREAM_PLAN_NODE_ID, downStreamPlanNodeIds); + } + // TODO implement pipeline division for shuffle sink context.setDegreeOfParallelism(1); List children = dealWithConsumeAllChildrenPipelineBreaker(node, context); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java index 1bc788251a764..74e6b3eb0f2d2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java @@ -38,6 +38,7 @@ import org.apache.iotdb.db.queryengine.execution.exchange.MPPDataExchangeManager; import org.apache.iotdb.db.queryengine.execution.exchange.MPPDataExchangeService; import org.apache.iotdb.db.queryengine.execution.exchange.sink.DownStreamChannelIndex; +import org.apache.iotdb.db.queryengine.execution.exchange.sink.DownStreamChannelLocation; import org.apache.iotdb.db.queryengine.execution.exchange.sink.ISinkHandle; import org.apache.iotdb.db.queryengine.execution.exchange.sink.ShuffleSinkHandle; import org.apache.iotdb.db.queryengine.execution.exchange.source.ISourceHandle; @@ -321,6 +322,8 @@ import static org.apache.iotdb.db.queryengine.execution.operator.process.join.merge.MergeSortComparator.getComparatorForTable; import static org.apache.iotdb.db.queryengine.execution.operator.process.rowpattern.PhysicalValuePointer.CLASSIFIER; import static org.apache.iotdb.db.queryengine.execution.operator.process.rowpattern.PhysicalValuePointer.MATCH_NUMBER; +import static org.apache.iotdb.db.queryengine.execution.operator.sink.IdentitySinkOperator.DELIMITER_BETWEEN_ID; +import static org.apache.iotdb.db.queryengine.execution.operator.sink.IdentitySinkOperator.DOWNSTREAM_PLAN_NODE_ID; import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.AbstractTableScanOperator.constructAlignedPath; import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.InformationSchemaContentSupplierFactory.getSupplier; import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.AccumulatorFactory.createAccumulator; @@ -388,7 +391,13 @@ public Operator visitIdentitySink(IdentitySinkNode node, LocalExecutionPlanConte context.getNextOperatorId(), node.getPlanNodeId(), IdentitySinkOperator.class.getSimpleName()); - + String downStreamPlanNodeId = + node.getDownStreamChannelLocationList().stream() + .map(DownStreamChannelLocation::getRemotePlanNodeId) + .collect(Collectors.joining(DELIMITER_BETWEEN_ID)); + if (!downStreamPlanNodeId.isEmpty()) { + operatorContext.recordSpecifiedInfo(DOWNSTREAM_PLAN_NODE_ID, downStreamPlanNodeId); + } checkArgument( MPP_DATA_EXCHANGE_MANAGER != null, "MPP_DATA_EXCHANGE_MANAGER should not be null"); FragmentInstanceId localInstanceId = context.getInstanceContext().getId(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/DistributionPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/DistributionPlanner.java index 261117b931bb5..ae17fdde04f0d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/DistributionPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/DistributionPlanner.java @@ -41,8 +41,10 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.sink.IdentitySinkNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.sink.MultiChildrenSinkNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.sink.ShuffleSinkNode; +import org.apache.iotdb.db.queryengine.plan.statement.Statement; import org.apache.iotdb.db.queryengine.plan.statement.component.OrderByComponent; import org.apache.iotdb.db.queryengine.plan.statement.crud.QueryStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.ExplainAnalyzeStatement; import org.apache.tsfile.external.commons.lang3.Validate; @@ -111,9 +113,7 @@ private void adjustUpStream(PlanNode root, NodeGroupContext context) { return; } - final boolean needShuffleSinkNode = - analysis.getTreeStatement() instanceof QueryStatement - && needShuffleSinkNode((QueryStatement) analysis.getTreeStatement(), context); + final boolean needShuffleSinkNode = needShuffleSinkNode(analysis.getTreeStatement(), context); adjustUpStreamHelper(root, new HashMap<>(), needShuffleSinkNode, context); } @@ -162,8 +162,18 @@ private void adjustUpStreamHelper( } /** Return true if we need to use ShuffleSinkNode instead of IdentitySinkNode. */ - private boolean needShuffleSinkNode( - QueryStatement queryStatement, NodeGroupContext nodeGroupContext) { + private boolean needShuffleSinkNode(Statement statement, NodeGroupContext nodeGroupContext) { + + QueryStatement queryStatement = null; + if (statement instanceof QueryStatement) { + queryStatement = (QueryStatement) statement; + } else if (statement instanceof ExplainAnalyzeStatement) { + queryStatement = ((ExplainAnalyzeStatement) statement).getQueryStatement(); + } + if (queryStatement == null) { + return false; + } + OrderByComponent orderByComponent = queryStatement.getOrderByComponent(); if (nodeGroupContext.isAlignByDevice() && orderByComponent != null) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/NodeGroupContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/NodeGroupContext.java index 5f01e646b06a5..ee8c0bb4c6f18 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/NodeGroupContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/NodeGroupContext.java @@ -29,6 +29,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.Statement; import org.apache.iotdb.db.queryengine.plan.statement.crud.QueryStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.ShowTimeSeriesStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.ExplainAnalyzeStatement; import java.util.Collections; import java.util.HashMap; @@ -50,6 +51,10 @@ public NodeGroupContext(MPPQueryContext queryContext, Statement statement, PlanN this.mostlyUsedDataRegion = getMostlyUsedDataRegion(root); } else if (statement instanceof ShowTimeSeriesStatement) { this.mostlyUsedDataRegion = getMostlyUsedDataRegion(root); + } else if (statement instanceof ExplainAnalyzeStatement) { + QueryStatement queryStatement = ((ExplainAnalyzeStatement) statement).getQueryStatement(); + this.isAlignByDevice = queryStatement.isAlignByDevice(); + this.mostlyUsedDataRegion = getMostlyUsedDataRegion(root); } this.hasExchangeNode = false; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SimpleFragmentParallelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SimpleFragmentParallelPlanner.java index ce953ecac9c77..add18b49a0d7c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SimpleFragmentParallelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SimpleFragmentParallelPlanner.java @@ -145,7 +145,8 @@ private void produceFragmentInstance(PlanFragment fragment) { queryContext.getSession(), queryContext.isExplainAnalyze(), queryContext.isDebug(), - fragment.isRoot()); + fragment.isRoot(), + queryContext.isVerbose()); selectExecutorAndHost( fragment, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/WriteFragmentParallelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/WriteFragmentParallelPlanner.java index a5d4a33cac011..edce982fea086 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/WriteFragmentParallelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/WriteFragmentParallelPlanner.java @@ -82,6 +82,7 @@ public List parallelPlan() { // Never timeout for write Long.MAX_VALUE, queryContext.getSession(), + false, false); if (split.getRegionReplicaSet() != null) { final TRegionReplicaSet validSet = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/FragmentInstance.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/FragmentInstance.java index 6bf584145cf81..d53e29e16924f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/FragmentInstance.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/FragmentInstance.java @@ -80,6 +80,7 @@ public class FragmentInstance implements IConsensusRequest { private boolean isExplainAnalyze = false; private final boolean debug; + private final boolean verbose; // We can add some more params for a specific FragmentInstance // So that we can make different FragmentInstance owns different data range. @@ -91,7 +92,8 @@ public FragmentInstance( QueryType type, long timeOut, SessionInfo sessionInfo, - boolean debug) { + boolean debug, + boolean verbose) { this.fragment = fragment; this.globalTimePredicate = globalTimePredicate; this.id = id; @@ -100,6 +102,7 @@ public FragmentInstance( this.isRoot = false; this.sessionInfo = sessionInfo; this.debug = debug; + this.verbose = verbose; } public FragmentInstance( @@ -111,8 +114,9 @@ public FragmentInstance( SessionInfo sessionInfo, boolean isExplainAnalyze, boolean debug, - boolean isRoot) { - this(fragment, id, globalTimePredicate, type, timeOut, sessionInfo, debug); + boolean isRoot, + boolean verbose) { + this(fragment, id, globalTimePredicate, type, timeOut, sessionInfo, debug, verbose); this.isRoot = isRoot; this.isExplainAnalyze = isExplainAnalyze; } @@ -125,8 +129,9 @@ public FragmentInstance( SessionInfo sessionInfo, boolean isExplainAnalyze, boolean debug, - boolean isRoot) { - this(fragment, id, null, type, timeOut, sessionInfo, debug); + boolean isRoot, + boolean verbose) { + this(fragment, id, null, type, timeOut, sessionInfo, debug, verbose); this.isRoot = isRoot; this.isExplainAnalyze = isExplainAnalyze; } @@ -139,8 +144,9 @@ public FragmentInstance( long timeOut, SessionInfo sessionInfo, int dataNodeFINum, - boolean debug) { - this(fragment, id, globalTimePredicate, type, timeOut, sessionInfo, debug); + boolean debug, + boolean verbose) { + this(fragment, id, globalTimePredicate, type, timeOut, sessionInfo, debug, verbose); this.dataNodeFINum = dataNodeFINum; } @@ -211,6 +217,10 @@ public boolean isDebug() { return debug; } + public boolean isVerbose() { + return verbose; + } + public String toString() { StringBuilder ret = new StringBuilder(); ret.append(String.format("FragmentInstance-%s:", getId())); @@ -241,6 +251,7 @@ public static FragmentInstance deserializeFrom(ByteBuffer buffer) { QueryType queryType = QueryType.values()[ReadWriteIOUtils.readInt(buffer)]; int dataNodeFINum = ReadWriteIOUtils.readInt(buffer); boolean debug = ReadWriteIOUtils.readBool(buffer); + boolean verbose = ReadWriteIOUtils.readBool(buffer); FragmentInstance fragmentInstance = new FragmentInstance( planFragment, @@ -250,7 +261,8 @@ public static FragmentInstance deserializeFrom(ByteBuffer buffer) { timeOut, sessionInfo, dataNodeFINum, - debug); + debug, + verbose); boolean hasHostDataNode = ReadWriteIOUtils.readBool(buffer); fragmentInstance.hostDataNode = hasHostDataNode ? ThriftCommonsSerDeUtils.deserializeTDataNodeLocation(buffer) : null; @@ -275,6 +287,7 @@ public ByteBuffer serializeToByteBuffer() { ReadWriteIOUtils.write(type.ordinal(), outputStream); ReadWriteIOUtils.write(dataNodeFINum, outputStream); ReadWriteIOUtils.write(debug, outputStream); + ReadWriteIOUtils.write(verbose, outputStream); ReadWriteIOUtils.write(hostDataNode != null, outputStream); if (hostDataNode != null) { ThriftCommonsSerDeUtils.serializeTDataNodeLocation(hostDataNode, outputStream); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java index 4f26c9ada04a0..03eb0baa00538 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializer.java @@ -143,7 +143,8 @@ public CteDataStore fetchCteQueryResult( context.getExplainType(), context.getTimeOut(), false, - context.isDebug()); + context.isDebug(), + context.isVerbose()); if (executionResult.status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { return null; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableModelQueryFragmentPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableModelQueryFragmentPlanner.java index 8a5ce92714031..6ad998a5a5335 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableModelQueryFragmentPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableModelQueryFragmentPlanner.java @@ -184,7 +184,8 @@ private void produceFragmentInstance( queryContext.getSession(), queryContext.isExplainAnalyze(), queryContext.isDebug(), - fragment.isRoot()); + fragment.isRoot(), + queryContext.isVerbose()); selectExecutorAndHost( fragment, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java index 17bc451c74245..49d4b7831e7d3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/ir/PredicateWithUncorrelatedScalarSubqueryReconstructor.java @@ -154,7 +154,8 @@ public Optional fetchUncorrelatedSubqueryResultForPredicate( ExplainType.NONE, context.getTimeOut(), false, - q.isDebug()); + q.isDebug(), + context.isVerbose()); // This may occur when the subquery cannot be executed in advance (for example, with // correlated scalar subqueries). diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java index 98d683284119c..1730fa2d921b7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java @@ -348,7 +348,8 @@ private boolean dispatchOnePieceNode( queryContext.getQueryType(), queryContext.getTimeOut() - (System.currentTimeMillis() - queryContext.getStartTime()), queryContext.getSession(), - queryContext.isDebug()); + queryContext.isDebug(), + queryContext.isVerbose()); instance.setExecutorAndHost(new StorageExecutor(replicaSet)); Future dispatchResultFuture = dispatcher.dispatch(null, Collections.singletonList(instance)); @@ -502,7 +503,8 @@ private boolean loadLocally(LoadSingleTsFileNode node) throws IoTDBException { queryContext.getTimeOut() - (System.currentTimeMillis() - queryContext.getStartTime()), queryContext.getSession(), - queryContext.isDebug()); + queryContext.isDebug(), + queryContext.isVerbose()); instance.setExecutorAndHost(new StorageExecutor(node.getLocalRegionReplicaSet())); dispatcher.dispatchLocally(instance); } catch (FragmentInstanceDispatchException e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java index c875d6007cff0..5e45c080ae3c9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java @@ -151,9 +151,8 @@ public List renderFragmentInstances( "ready queued time: %.3f ms, blocked queued time: %.3f ms", statistics.getReadyQueuedTime() * NS_TO_MS_FACTOR, statistics.getBlockQueuedTime() * NS_TO_MS_FACTOR)); - if (verbose) { - renderQueryStatistics(statistics.getQueryStatistics(), singleFragmentInstanceArea); - } + renderQueryStatistics(statistics.getQueryStatistics(), singleFragmentInstanceArea, verbose); + // render operator PlanNode planNodeTree = instance.getFragment().getPlanNodeTree(); renderOperator( @@ -193,258 +192,288 @@ private void addBlankLine(List singleFragmentInstanceArea) { } private void renderQueryStatistics( - TQueryStatistics queryStatistics, List singleFragmentInstanceArea) { + TQueryStatistics queryStatistics, + List singleFragmentInstanceArea, + boolean verbose) { addLine(singleFragmentInstanceArea, 1, "Query Statistics:"); - addLineWithoutValueCheck( - singleFragmentInstanceArea, - 2, - "loadBloomFilterFromCacheCount", - queryStatistics.loadBloomFilterFromCacheCount); - addLineWithoutValueCheck( - singleFragmentInstanceArea, - 2, - "loadBloomFilterFromDiskCount", - queryStatistics.loadBloomFilterFromDiskCount); - addLineWithoutValueCheck( - singleFragmentInstanceArea, - 2, - "loadBloomFilterActualIOSize", - queryStatistics.loadBloomFilterActualIOSize); - addLineWithoutValueCheck( - singleFragmentInstanceArea, - 2, - "loadBloomFilterTime", - queryStatistics.loadBloomFilterTime * NS_TO_MS_FACTOR); + if (verbose) { + addLineWithoutValueCheck( + singleFragmentInstanceArea, + 2, + "loadBloomFilterFromCacheCount", + queryStatistics.loadBloomFilterFromCacheCount); + addLineWithoutValueCheck( + singleFragmentInstanceArea, + 2, + "loadBloomFilterFromDiskCount", + queryStatistics.loadBloomFilterFromDiskCount); + addLineWithoutValueCheck( + singleFragmentInstanceArea, + 2, + "loadBloomFilterActualIOSize", + queryStatistics.loadBloomFilterActualIOSize); + addLineWithoutValueCheck( + singleFragmentInstanceArea, + 2, + "loadBloomFilterTime", + queryStatistics.loadBloomFilterTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataDiskSeqCount", - queryStatistics.loadTimeSeriesMetadataDiskSeqCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataDiskUnSeqCount", - queryStatistics.loadTimeSeriesMetadataDiskUnSeqCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataMemSeqCount", - queryStatistics.loadTimeSeriesMetadataMemSeqCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataMemUnSeqCount", - queryStatistics.loadTimeSeriesMetadataMemUnSeqCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataAlignedDiskSeqCount", - queryStatistics.loadTimeSeriesMetadataAlignedDiskSeqCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataAlignedDiskUnSeqCount", - queryStatistics.loadTimeSeriesMetadataAlignedDiskUnSeqCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataAlignedMemSeqCount", - queryStatistics.loadTimeSeriesMetadataAlignedMemSeqCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataAlignedMemUnSeqCount", - queryStatistics.loadTimeSeriesMetadataAlignedMemUnSeqCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataDiskSeqCount", + queryStatistics.loadTimeSeriesMetadataDiskSeqCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataDiskUnSeqCount", + queryStatistics.loadTimeSeriesMetadataDiskUnSeqCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataMemSeqCount", + queryStatistics.loadTimeSeriesMetadataMemSeqCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataMemUnSeqCount", + queryStatistics.loadTimeSeriesMetadataMemUnSeqCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataAlignedDiskSeqCount", + queryStatistics.loadTimeSeriesMetadataAlignedDiskSeqCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataAlignedDiskUnSeqCount", + queryStatistics.loadTimeSeriesMetadataAlignedDiskUnSeqCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataAlignedMemSeqCount", + queryStatistics.loadTimeSeriesMetadataAlignedMemSeqCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataAlignedMemUnSeqCount", + queryStatistics.loadTimeSeriesMetadataAlignedMemUnSeqCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataDiskSeqTime", - queryStatistics.loadTimeSeriesMetadataDiskSeqTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataDiskUnSeqTime", - queryStatistics.loadTimeSeriesMetadataDiskUnSeqTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataMemSeqTime", - queryStatistics.loadTimeSeriesMetadataMemSeqTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataMemUnSeqTime", - queryStatistics.loadTimeSeriesMetadataMemUnSeqTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataAlignedDiskSeqTime", - queryStatistics.loadTimeSeriesMetadataAlignedDiskSeqTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataAlignedDiskUnSeqTime", - queryStatistics.loadTimeSeriesMetadataAlignedDiskUnSeqTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataAlignedMemSeqTime", - queryStatistics.loadTimeSeriesMetadataAlignedMemSeqTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataAlignedMemUnSeqTime", - queryStatistics.loadTimeSeriesMetadataAlignedMemUnSeqTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataDiskSeqTime", + queryStatistics.loadTimeSeriesMetadataDiskSeqTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataDiskUnSeqTime", + queryStatistics.loadTimeSeriesMetadataDiskUnSeqTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataMemSeqTime", + queryStatistics.loadTimeSeriesMetadataMemSeqTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataMemUnSeqTime", + queryStatistics.loadTimeSeriesMetadataMemUnSeqTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataAlignedDiskSeqTime", + queryStatistics.loadTimeSeriesMetadataAlignedDiskSeqTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataAlignedDiskUnSeqTime", + queryStatistics.loadTimeSeriesMetadataAlignedDiskUnSeqTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataAlignedMemSeqTime", + queryStatistics.loadTimeSeriesMetadataAlignedMemSeqTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataAlignedMemUnSeqTime", + queryStatistics.loadTimeSeriesMetadataAlignedMemUnSeqTime * NS_TO_MS_FACTOR); - addLineWithoutValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataFromCacheCount", - queryStatistics.loadTimeSeriesMetadataFromCacheCount); - addLineWithoutValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataFromDiskCount", - queryStatistics.loadTimeSeriesMetadataFromDiskCount); - addLineWithoutValueCheck( - singleFragmentInstanceArea, - 2, - "loadTimeSeriesMetadataActualIOSize", - queryStatistics.loadTimeSeriesMetadataActualIOSize); + addLineWithoutValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataFromCacheCount", + queryStatistics.loadTimeSeriesMetadataFromCacheCount); + addLineWithoutValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataFromDiskCount", + queryStatistics.loadTimeSeriesMetadataFromDiskCount); + addLineWithoutValueCheck( + singleFragmentInstanceArea, + 2, + "loadTimeSeriesMetadataActualIOSize", + queryStatistics.loadTimeSeriesMetadataActualIOSize); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "alignedTimeSeriesMetadataModificationCount", - queryStatistics.getAlignedTimeSeriesMetadataModificationCount()); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "alignedTimeSeriesMetadataModificationTime", - queryStatistics.getAlignedTimeSeriesMetadataModificationTime() * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "nonAlignedTimeSeriesMetadataModificationCount", - queryStatistics.getNonAlignedTimeSeriesMetadataModificationCount()); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "nonAlignedTimeSeriesMetadataModificationTime", - queryStatistics.getNonAlignedTimeSeriesMetadataModificationTime() * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "alignedTimeSeriesMetadataModificationCount", + queryStatistics.getAlignedTimeSeriesMetadataModificationCount()); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "alignedTimeSeriesMetadataModificationTime", + queryStatistics.getAlignedTimeSeriesMetadataModificationTime() * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "nonAlignedTimeSeriesMetadataModificationCount", + queryStatistics.getNonAlignedTimeSeriesMetadataModificationCount()); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "nonAlignedTimeSeriesMetadataModificationTime", + queryStatistics.getNonAlignedTimeSeriesMetadataModificationTime() * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "constructNonAlignedChunkReadersDiskCount", - queryStatistics.constructNonAlignedChunkReadersDiskCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "constructNonAlignedChunkReadersMemCount", - queryStatistics.constructNonAlignedChunkReadersMemCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "constructAlignedChunkReadersDiskCount", - queryStatistics.constructAlignedChunkReadersDiskCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "constructAlignedChunkReadersMemCount", - queryStatistics.constructAlignedChunkReadersMemCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "constructNonAlignedChunkReadersDiskCount", + queryStatistics.constructNonAlignedChunkReadersDiskCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "constructNonAlignedChunkReadersMemCount", + queryStatistics.constructNonAlignedChunkReadersMemCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "constructAlignedChunkReadersDiskCount", + queryStatistics.constructAlignedChunkReadersDiskCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "constructAlignedChunkReadersMemCount", + queryStatistics.constructAlignedChunkReadersMemCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "constructNonAlignedChunkReadersDiskTime", - queryStatistics.constructNonAlignedChunkReadersDiskTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "constructNonAlignedChunkReadersMemTime", - queryStatistics.constructNonAlignedChunkReadersMemTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "constructAlignedChunkReadersDiskTime", - queryStatistics.constructAlignedChunkReadersDiskTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "constructAlignedChunkReadersMemTime", - queryStatistics.constructAlignedChunkReadersMemTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "constructNonAlignedChunkReadersDiskTime", + queryStatistics.constructNonAlignedChunkReadersDiskTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "constructNonAlignedChunkReadersMemTime", + queryStatistics.constructNonAlignedChunkReadersMemTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "constructAlignedChunkReadersDiskTime", + queryStatistics.constructAlignedChunkReadersDiskTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "constructAlignedChunkReadersMemTime", + queryStatistics.constructAlignedChunkReadersMemTime * NS_TO_MS_FACTOR); + + addLineWithoutValueCheck( + singleFragmentInstanceArea, + 2, + "loadChunkFromCacheCount", + queryStatistics.loadChunkFromCacheCount); + addLineWithoutValueCheck( + singleFragmentInstanceArea, + 2, + "loadChunkFromDiskCount", + queryStatistics.loadChunkFromDiskCount); + addLineWithoutValueCheck( + singleFragmentInstanceArea, + 2, + "loadChunkActualIOSize", + queryStatistics.loadChunkActualIOSize); + + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "pageReadersDecodeAlignedDiskCount", + queryStatistics.pageReadersDecodeAlignedDiskCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "pageReadersDecodeAlignedDiskTime", + queryStatistics.pageReadersDecodeAlignedDiskTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "pageReadersDecodeAlignedMemCount", + queryStatistics.pageReadersDecodeAlignedMemCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "pageReadersDecodeAlignedMemTime", + queryStatistics.pageReadersDecodeAlignedMemTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "pageReadersDecodeNonAlignedDiskCount", + queryStatistics.pageReadersDecodeNonAlignedDiskCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "pageReadersDecodeNonAlignedDiskTime", + queryStatistics.pageReadersDecodeNonAlignedDiskTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "pageReadersDecodeNonAlignedMemCount", + queryStatistics.pageReadersDecodeNonAlignedMemCount); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "pageReadersDecodeNonAlignedMemTime", + queryStatistics.pageReadersDecodeNonAlignedMemTime * NS_TO_MS_FACTOR); + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "pageReaderMaxUsedMemorySize", + queryStatistics.pageReaderMaxUsedMemorySize); + + addLineWithValueCheck( + singleFragmentInstanceArea, + 2, + "chunkWithMetadataErrorsCount", + queryStatistics.chunkWithMetadataErrorsCount); + } addLineWithoutValueCheck( singleFragmentInstanceArea, 2, - "loadChunkFromCacheCount", - queryStatistics.loadChunkFromCacheCount); - addLineWithoutValueCheck( - singleFragmentInstanceArea, - 2, - "loadChunkFromDiskCount", - queryStatistics.loadChunkFromDiskCount); + "timeSeriesIndexFilteredRows", + queryStatistics.timeSeriesIndexFilteredRows); + addLineWithoutValueCheck( singleFragmentInstanceArea, 2, - "loadChunkActualIOSize", - queryStatistics.loadChunkActualIOSize); + "chunkIndexFilteredRows", + queryStatistics.chunkIndexFilteredRows); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "pageReadersDecodeAlignedDiskCount", - queryStatistics.pageReadersDecodeAlignedDiskCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "pageReadersDecodeAlignedDiskTime", - queryStatistics.pageReadersDecodeAlignedDiskTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "pageReadersDecodeAlignedMemCount", - queryStatistics.pageReadersDecodeAlignedMemCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "pageReadersDecodeAlignedMemTime", - queryStatistics.pageReadersDecodeAlignedMemTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "pageReadersDecodeNonAlignedDiskCount", - queryStatistics.pageReadersDecodeNonAlignedDiskCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "pageReadersDecodeNonAlignedDiskTime", - queryStatistics.pageReadersDecodeNonAlignedDiskTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "pageReadersDecodeNonAlignedMemCount", - queryStatistics.pageReadersDecodeNonAlignedMemCount); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "pageReadersDecodeNonAlignedMemTime", - queryStatistics.pageReadersDecodeNonAlignedMemTime * NS_TO_MS_FACTOR); - addLineWithValueCheck( + addLineWithoutValueCheck( singleFragmentInstanceArea, 2, - "pageReaderMaxUsedMemorySize", - queryStatistics.pageReaderMaxUsedMemorySize); + "pageIndexFilteredRows", + queryStatistics.pageIndexFilteredRows); - addLineWithValueCheck( - singleFragmentInstanceArea, - 2, - "chunkWithMetadataErrorsCount", - queryStatistics.chunkWithMetadataErrorsCount); + if (verbose) { + addLineWithoutValueCheck( + singleFragmentInstanceArea, + 2, + "rowScanFilteredRows", + queryStatistics.rowScanFilteredRows); + } } private void addLine(List resultForSingleInstance, int level, String value) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index 89fe9facc71bc..517566ed78c8f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -517,6 +517,7 @@ public MemPointIterator createMemPointIterator(Ordering scanOrder, Filter global floatPrecision, encodingList, context.isIgnoreAllNullRows(), - MAX_NUMBER_OF_POINTS_IN_PAGE); + MAX_NUMBER_OF_POINTS_IN_PAGE, + context); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 86878ea8dc9fa..7c55b4a09dddd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -800,7 +800,8 @@ public void encode(BlockingQueue ioTaskQueue, BatchEncodeInfo encodeInfo columnIndexList, alignedTvLists, ignoreAllNullRows, - encodeInfo.maxNumberOfPointsInPage); + encodeInfo.maxNumberOfPointsInPage, + null); while (timeValuePairIterator.hasNextBatch()) { timeValuePairIterator.encodeBatch(alignedChunkWriter, encodeInfo, times); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index 5ea4e67ee7966..f6fa9403c7b56 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -423,6 +423,7 @@ public MemPointIterator createMemPointIterator(Ordering scanOrder, Filter global deletionList, floatPrecision, encoding, - MAX_NUMBER_OF_POINTS_IN_PAGE); + MAX_NUMBER_OF_POINTS_IN_PAGE, + context); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index 571a371c84957..e778e248b1daf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -487,7 +487,7 @@ public void encode(BlockingQueue ioTaskQueue, BatchEncodeInfo encodeInfo tvLists.add(workingListForFlush); MemPointIterator timeValuePairIterator = MemPointIteratorFactory.create( - schema.getType(), tvLists, encodeInfo.maxNumberOfPointsInPage); + schema.getType(), tvLists, encodeInfo.maxNumberOfPointsInPage, null); while (timeValuePairIterator.hasNextBatch()) { timeValuePairIterator.encodeBatch(chunkWriterImpl, encodeInfo, times); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java index 27883b34e9ce8..dff789bb98501 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java @@ -40,6 +40,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.function.LongConsumer; import static org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet.INIT_CHUNK_READER_ALIGNED_DISK; @@ -118,12 +119,15 @@ public IChunkReader getChunkReader(IChunkMetadata chunkMetaData, Filter globalTi } valueChunkList.add(chunk); } - + LongConsumer filterRowsRecorder = + this.context.getQueryStatistics()::addFilteredRowsOfPageLevel; long t2 = System.nanoTime(); IChunkReader chunkReader = ignoreAllNullRows - ? new AlignedChunkReader(timeChunk, valueChunkList, globalTimeFilter) - : new TableChunkReader(timeChunk, valueChunkList, globalTimeFilter); + ? new AlignedChunkReader( + timeChunk, valueChunkList, globalTimeFilter, filterRowsRecorder) + : new TableChunkReader( + timeChunk, valueChunkList, globalTimeFilter, filterRowsRecorder); SERIES_SCAN_COST_METRIC_SET.recordSeriesScanCost( INIT_CHUNK_READER_ALIGNED_DISK, System.nanoTime() - t2); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java index dd595f9eff4a4..da0779129c754 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java @@ -103,7 +103,11 @@ public IChunkReader getChunkReader(IChunkMetadata chunkMetaData, Filter globalTi } long t2 = System.nanoTime(); - IChunkReader chunkReader = new ChunkReader(chunk, globalTimeFilter); + IChunkReader chunkReader = + new ChunkReader( + chunk, + globalTimeFilter, + this.context.getQueryStatistics()::addFilteredRowsOfPageLevel); SeriesScanCostMetricSet.getInstance() .recordSeriesScanCost(INIT_CHUNK_READER_NONALIGNED_DISK, System.nanoTime() - t2); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java index 758721f9a05bc..56c04f05ec813 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedPageReader.java @@ -44,6 +44,7 @@ import java.util.Arrays; import java.util.List; import java.util.Optional; +import java.util.function.LongConsumer; import java.util.function.Supplier; import static org.apache.tsfile.read.reader.series.PaginationController.UNLIMITED_PAGINATION_CONTROLLER; @@ -90,7 +91,7 @@ public BatchData getAllSatisfiedPageData(boolean ascending) throws IOException { BatchData batchData = BatchDataFactory.createBatchData(TSDataType.VECTOR, ascending, false); - boolean[] satisfyInfo = buildSatisfyInfoArray(); + boolean[] satisfyInfo = buildSatisfyInfoArray(null); for (int rowIndex = 0; rowIndex < tsBlock.getPositionCount(); rowIndex++) { if (satisfyInfo[rowIndex]) { @@ -109,11 +110,16 @@ public BatchData getAllSatisfiedPageData(boolean ascending) throws IOException { @Override public TsBlock getAllSatisfiedData() { + return getAllSatisfiedData(null); + } + + @Override + public TsBlock getAllSatisfiedData(LongConsumer filterRowsRecorder) { getTsBlock(); builder.reset(); - boolean[] satisfyInfo = buildSatisfyInfoArray(); + boolean[] satisfyInfo = buildSatisfyInfoArray(filterRowsRecorder); // build time column int readEndIndex = buildTimeColumn(satisfyInfo); @@ -128,13 +134,19 @@ public TsBlock getAllSatisfiedData() { private static final Logger LOGGER = LoggerFactory.getLogger(MemAlignedPageReader.class); - private boolean[] buildSatisfyInfoArray() { + private boolean[] buildSatisfyInfoArray(LongConsumer filterRowsRecorder) { if (recordFilter == null || recordFilter.allSatisfy(this)) { boolean[] satisfyInfo = new boolean[tsBlock.getPositionCount()]; Arrays.fill(satisfyInfo, true); return satisfyInfo; } - return recordFilter.satisfyTsBlock(tsBlock); + if (filterRowsRecorder == null) { + return recordFilter.satisfyTsBlock(tsBlock); + } else { + boolean[] selection = new boolean[tsBlock.getPositionCount()]; + Arrays.fill(selection, true); + return recordFilter.satisfyTsBlock(selection, tsBlock, filterRowsRecorder); + } } private int buildTimeColumn(boolean[] satisfyInfo) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java index 0f656a96f691a..911aa9c21afa6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemPageReader.java @@ -41,6 +41,7 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.function.LongConsumer; import java.util.function.Supplier; import static org.apache.tsfile.read.reader.series.PaginationController.UNLIMITED_PAGINATION_CONTROLLER; @@ -79,8 +80,7 @@ public BatchData getAllSatisfiedPageData(boolean ascending) throws IOException { getTsBlock(); BatchData batchData = BatchDataFactory.createBatchData(tsDataType, ascending, false); - - boolean[] satisfyInfo = buildSatisfyInfoArray(); + boolean[] satisfyInfo = buildSatisfyInfoArray(null); for (int i = 0; i < tsBlock.getPositionCount(); i++) { if (satisfyInfo[i]) { @@ -122,11 +122,16 @@ public BatchData getAllSatisfiedPageData(boolean ascending) throws IOException { @Override public TsBlock getAllSatisfiedData() { + return getAllSatisfiedData(null); + } + + @Override + public TsBlock getAllSatisfiedData(LongConsumer filterRowsRecorder) { getTsBlock(); TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(tsDataType)); - boolean[] satisfyInfo = buildSatisfyInfoArray(); + boolean[] satisfyInfo = buildSatisfyInfoArray(filterRowsRecorder); // build time column int readEndIndex = buildTimeColumn(builder, satisfyInfo); @@ -137,13 +142,20 @@ public TsBlock getAllSatisfiedData() { return builder.build(); } - private boolean[] buildSatisfyInfoArray() { + private boolean[] buildSatisfyInfoArray(LongConsumer filterRowsRecorder) { if (recordFilter == null || recordFilter.allSatisfy(this)) { boolean[] satisfyInfo = new boolean[tsBlock.getPositionCount()]; Arrays.fill(satisfyInfo, true); return satisfyInfo; } - return recordFilter.satisfyTsBlock(tsBlock); + + if (filterRowsRecorder == null) { + return recordFilter.satisfyTsBlock(tsBlock); + } else { + boolean[] selection = new boolean[tsBlock.getPositionCount()]; + Arrays.fill(selection, true); + return recordFilter.satisfyTsBlock(selection, tsBlock, filterRowsRecorder); + } } private int buildTimeColumn(TsBlockBuilder builder, boolean[] satisfyInfo) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/ChunkMetadataLoaderUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/ChunkMetadataLoaderUtils.java new file mode 100644 index 0000000000000..96cb7c8c14bfc --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/ChunkMetadataLoaderUtils.java @@ -0,0 +1,55 @@ +/* + * 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.iotdb.db.storageengine.dataregion.read.reader.chunk.metadata; + +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; + +import org.apache.tsfile.file.metadata.IChunkMetadata; +import org.apache.tsfile.read.filter.basic.Filter; + +public class ChunkMetadataLoaderUtils { + + private ChunkMetadataLoaderUtils() {} + + /** + * Checks whether a chunk should be skipped during query execution. * + * + *

A chunk is skipped if: + * + *

    + *
  1. Its time range is invalid ({@code startTime > endTime}). + *
  2. The global time filter can fully eliminate it based on statistics, in which case the + * chunk's row count is recorded in filtered-rows statistics at chunk level. + *
+ * + *

Note: for unclosed TsFiles, {@link IChunkMetadata} may return an inaccurate row count. + */ + public static boolean shouldSkipAndRecord( + IChunkMetadata metadata, Filter globalTimeFilter, QueryContext context) { + if (metadata.getStartTime() > metadata.getEndTime()) { + return true; + } + if (globalTimeFilter != null && globalTimeFilter.canSkip(metadata)) { + context.getQueryStatistics().addFilteredRowsOfChunkLevel(metadata.getStatistics().getCount()); + return true; + } + return false; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/DiskAlignedChunkMetadataLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/DiskAlignedChunkMetadataLoader.java index f243b51ee1a79..64244677c6e07 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/DiskAlignedChunkMetadataLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/DiskAlignedChunkMetadataLoader.java @@ -93,9 +93,8 @@ public List loadChunkMetadataList(ITimeSeriesMetadata timeSeries final long t2 = System.nanoTime(); alignedChunkMetadataList.removeIf( alignedChunkMetaData -> - (globalTimeFilter != null && globalTimeFilter.canSkip(alignedChunkMetaData)) - || alignedChunkMetaData.getStartTime() > alignedChunkMetaData.getEndTime()); - + ChunkMetadataLoaderUtils.shouldSkipAndRecord( + alignedChunkMetaData, globalTimeFilter, context)); if (context.isDebug()) { DEBUG_LOGGER.info("After removed by filter Chunk meta data list is: "); alignedChunkMetadataList.forEach(c -> DEBUG_LOGGER.info(c.toString())); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/DiskChunkMetadataLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/DiskChunkMetadataLoader.java index f7b4790e0f75e..1517ce5feb578 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/DiskChunkMetadataLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/DiskChunkMetadataLoader.java @@ -79,8 +79,8 @@ public List loadChunkMetadataList(ITimeSeriesMetadata timeSeries final long t2 = System.nanoTime(); chunkMetadataList.removeIf( chunkMetaData -> - (globalTimeFilter != null && globalTimeFilter.canSkip(chunkMetaData)) - || chunkMetaData.getStartTime() > chunkMetaData.getEndTime()); + ChunkMetadataLoaderUtils.shouldSkipAndRecord( + chunkMetaData, globalTimeFilter, context)); if (context.isDebug()) { DEBUG_LOGGER.info("After removed by filter Chunk meta data list is: "); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/MemAlignedChunkMetadataLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/MemAlignedChunkMetadataLoader.java index 5f88b1cfb4f76..f075fb6eeee26 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/MemAlignedChunkMetadataLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/MemAlignedChunkMetadataLoader.java @@ -99,8 +99,8 @@ public List loadChunkMetadataList(ITimeSeriesMetadata timeSeries long t2 = System.nanoTime(); chunkMetadataList.removeIf( chunkMetaData -> - (globalTimeFilter != null && globalTimeFilter.canSkip(chunkMetaData)) - || chunkMetaData.getStartTime() > chunkMetaData.getEndTime()); + ChunkMetadataLoaderUtils.shouldSkipAndRecord( + chunkMetaData, globalTimeFilter, context)); SERIES_SCAN_COST_METRIC_SET.recordSeriesScanCost( CHUNK_METADATA_FILTER_ALIGNED_MEM, System.nanoTime() - t2); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/MemChunkMetadataLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/MemChunkMetadataLoader.java index 11bc4fb97362d..b5308bf419ca6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/MemChunkMetadataLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/metadata/MemChunkMetadataLoader.java @@ -92,8 +92,8 @@ public List loadChunkMetadataList(ITimeSeriesMetadata timeSeries long t2 = System.nanoTime(); chunkMetadataList.removeIf( chunkMetaData -> - (globalTimeFilter != null && globalTimeFilter.canSkip(chunkMetaData)) - || chunkMetaData.getStartTime() > chunkMetaData.getEndTime()); + ChunkMetadataLoaderUtils.shouldSkipAndRecord( + chunkMetaData, globalTimeFilter, context)); SERIES_SCAN_COST_METRIC_SET.recordSeriesScanCost( CHUNK_METADATA_FILTER_NONALIGNED_MEM, System.nanoTime() - t2); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index f298a69eaf027..da4ae4ab21639 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -1034,7 +1034,8 @@ public boolean isSatisfied(IDeviceID deviceId, Filter timeFilter, boolean isSeq, endTime); return false; } - + // we cannot count the filtered rows in tsfile due to the lack of related info about the + // tsfile boolean res = timeFilter.satisfyStartEndTime(startTime, endTime); if (debug && !res) { DEBUG_LOGGER.info( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 2c87fdb32abec..6293a0fc4b94b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; @@ -1733,7 +1734,8 @@ public AlignedTVListIterator iterator( Integer floatPrecision, List encodingList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new AlignedTVListIterator( scanOrder, rowCount, @@ -1745,7 +1747,8 @@ public AlignedTVListIterator iterator( floatPrecision, encodingList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } /* AlignedTVList Iterator */ @@ -1777,8 +1780,17 @@ public AlignedTVListIterator( Integer floatPrecision, List encodingList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { - super(scanOrder, rowCount, globalTimeFilter, null, null, null, maxNumberOfPointsInPage); + int maxNumberOfPointsInPage, + QueryContext queryContext) { + super( + scanOrder, + rowCount, + globalTimeFilter, + null, + null, + null, + maxNumberOfPointsInPage, + queryContext); this.dataTypeList = dataTypeList; this.columnIndexList = (columnIndexList == null) @@ -1811,6 +1823,9 @@ public AlignedTVListIterator( protected void prepareNext() { // find the first row that is neither deleted nor empty (all NULL values) findValidRow = false; + long filteredRows = 0; + boolean isRecord = this.getQueryContext() != null && this.getQueryContext().isVerbose(); + while (index < rows && !findValidRow) { // all columns values are deleted int convertedScanOrderValueIndex = getValueIndex(getScanOrderIndex(index)); @@ -1821,15 +1836,22 @@ protected void prepareNext() { continue; } long time = getTime(getScanOrderIndex(index)); - if (isPointDeleted(time, timeColumnDeletion, timeDeleteCursor, scanOrder) - || !isTimeSatisfied(time)) { + if (isPointDeleted(time, timeColumnDeletion, timeDeleteCursor, scanOrder)) { + index++; + continue; + } + if (!isTimeSatisfied(time)) { index++; + filteredRows++; continue; } // does not find any valid row if (index >= rows) { probeNext = true; + if (isRecord) { + this.getQueryContext().getQueryStatistics().addFilteredRowsOfRowLevel(filteredRows); + } return; } // When traversing in ASC order, we only need to overwrite the previous non-null value @@ -1898,6 +1920,9 @@ && isPointDeleted( } } probeNext = true; + if (isRecord) { + this.getQueryContext().getQueryStatistics().addFilteredRowsOfRowLevel(filteredRows); + } } // When used as a point reader, we should not apply a pagination controller or push down filter @@ -2001,6 +2026,34 @@ public boolean hasNextBatch() { return index < rows && !isCurrentTimeExceedTimeRange(getTime(getScanOrderIndex(index))); } + private boolean isRowInvalid( + int rowIndex, long time, int[] deleteCursor, long[] filteredRowsByTimeFilter) { + if ((allValueColDeletedMap != null + && allValueColDeletedMap.isMarked(getValueIndex(getScanOrderIndex(rowIndex)))) + || isTimeDeleted(getScanOrderIndex(rowIndex)) + || isPointDeleted(time, timeColumnDeletion, deleteCursor, scanOrder)) { + return true; + } + + // only count the filtered rows when data is filtered only by Time filter + if (!isTimeSatisfied(time)) { + if (filteredRowsByTimeFilter != null) { + filteredRowsByTimeFilter[0]++; + } + return true; + } + return false; + } + + private LazyBitMap markRowAsInvalid( + LazyBitMap timeInvalidInfo, int rowIndex, int maxRowCountOfCurrentBatch) { + if (timeInvalidInfo == null) { + timeInvalidInfo = new LazyBitMap(rowIndex, maxRowCountOfCurrentBatch, rows - 1); + } + timeInvalidInfo.mark(rowIndex); + return timeInvalidInfo; + } + @Override public TsBlock nextBatch() { int maxRowCountOfCurrentBatch = Math.min(rows - index, maxNumberOfPointsInPage); @@ -2016,6 +2069,10 @@ public TsBlock nextBatch() { int[] deleteCursor = {0}; int startIndex = index; + + boolean isVerbose = this.getQueryContext().isVerbose(); + long[] filteredRowsByTimeFilter = this.getQueryContext().isVerbose() ? new long[] {0} : null; + // time column for (; index < rows; index++) { long time = getTime(getScanOrderIndex(index)); @@ -2023,33 +2080,19 @@ public TsBlock nextBatch() { break; } // skip invalid row - if ((allValueColDeletedMap != null - && allValueColDeletedMap.isMarked(getValueIndex(getScanOrderIndex(index)))) - || isTimeDeleted(getScanOrderIndex(index)) - || !isTimeSatisfied(time) - || isPointDeleted(time, timeColumnDeletion, deleteCursor, scanOrder)) { - timeInvalidInfo = - timeInvalidInfo == null - ? new LazyBitMap(index, maxRowCountOfCurrentBatch, rows - 1) - : timeInvalidInfo; - timeInvalidInfo.mark(index); + if (isRowInvalid(index, time, deleteCursor, filteredRowsByTimeFilter)) { + timeInvalidInfo = markRowAsInvalid(timeInvalidInfo, index, maxRowCountOfCurrentBatch); continue; } int nextRowIndex = index + 1; - long timeOfNextRowIndex; while (nextRowIndex < rows - && ((allValueColDeletedMap != null - && allValueColDeletedMap.isMarked( - getValueIndex(getScanOrderIndex(nextRowIndex)))) - || isTimeDeleted(getScanOrderIndex(nextRowIndex)) - || !isTimeSatisfied((timeOfNextRowIndex = getTime(getScanOrderIndex(nextRowIndex)))) - || isPointDeleted( - timeOfNextRowIndex, timeColumnDeletion, deleteCursor, scanOrder))) { + && (isRowInvalid( + nextRowIndex, + getTime(getScanOrderIndex(nextRowIndex)), + deleteCursor, + filteredRowsByTimeFilter))) { timeInvalidInfo = - timeInvalidInfo == null - ? new LazyBitMap(nextRowIndex, maxRowCountOfCurrentBatch, rows - 1) - : timeInvalidInfo; - timeInvalidInfo.mark(nextRowIndex); + markRowAsInvalid(timeInvalidInfo, nextRowIndex, maxRowCountOfCurrentBatch); nextRowIndex++; } if ((nextRowIndex == rows || time != getTime(getScanOrderIndex(nextRowIndex)))) { @@ -2066,6 +2109,12 @@ public TsBlock nextBatch() { } index = nextRowIndex - 1; } + // record the filtered rows by time filter + if (isVerbose && filteredRowsByTimeFilter != null && filteredRowsByTimeFilter[0] > 0) { + this.getQueryContext() + .getQueryStatistics() + .addFilteredRowsOfRowLevel(filteredRowsByTimeFilter[0]); + } boolean[] hasAnyNonNullValue = new boolean[validRowCount]; int columnCount = dataTypeList.size(); @@ -2163,15 +2212,25 @@ public TsBlock nextBatch() { if (ignoreAllNullRows && needRebuildTsBlock(hasAnyNonNullValue)) { // if exist all null rows, at most have validRowCount - 1 valid rows // When rebuilding TsBlock, pushDownFilter and paginationController are also processed. - tsBlock = reBuildTsBlock(hasAnyNonNullValue, validRowCount, dataTypeList, tsBlock); + tsBlock = + reBuildTsBlock(hasAnyNonNullValue, validRowCount, dataTypeList, tsBlock, isVerbose); } else if (pushDownFilter != null) { + TsBlockBuilder tsBlockBuilder = + new TsBlockBuilder( + Math.min(maxNumberOfPointsInPage, tsBlock.getPositionCount()), dataTypeList); + + // record the filtered rows if verbose is specified tsBlock = - TsBlockUtil.applyFilterAndLimitOffsetToTsBlock( - tsBlock, - new TsBlockBuilder( - Math.min(maxNumberOfPointsInPage, tsBlock.getPositionCount()), dataTypeList), - pushDownFilter, - paginationController); + isVerbose + ? TsBlockUtil.applyFilterAndLimitOffsetToTsBlock( + tsBlock, + tsBlockBuilder, + pushDownFilter, + paginationController, + s -> this.getQueryContext().getQueryStatistics().addFilteredRowsOfRowLevel(s)) + : TsBlockUtil.applyFilterAndLimitOffsetToTsBlock( + tsBlock, tsBlockBuilder, pushDownFilter, paginationController); + } else { tsBlock = paginationController.applyTsBlock(tsBlock); } @@ -2233,10 +2292,18 @@ private TsBlock reBuildTsBlock( boolean[] hasAnyNonNullValue, int previousValidRowCount, List tsDataTypeList, - TsBlock previousTsBlock) { + TsBlock previousTsBlock, + boolean isVerbose) { boolean[] selection = hasAnyNonNullValue; if (pushDownFilter != null) { - selection = pushDownFilter.satisfyTsBlock(hasAnyNonNullValue, previousTsBlock); + // record the filtered rows if verbose is specified + selection = + isVerbose + ? pushDownFilter.satisfyTsBlock( + hasAnyNonNullValue, + previousTsBlock, + s -> this.getQueryContext().getQueryStatistics().addFilteredRowsOfRowLevel(s)) + : pushDownFilter.satisfyTsBlock(hasAnyNonNullValue, previousTsBlock); } TsBlockBuilder builder = new TsBlockBuilder(previousValidRowCount - 1, tsDataTypeList); TimeColumnBuilder timeColumnBuilder = builder.getTimeColumnBuilder(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MemPointIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MemPointIterator.java index ca5686f7ac37d..088187b3b4230 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MemPointIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MemPointIterator.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.utils.datastructure; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.tsfile.read.common.TimeRange; @@ -43,9 +44,11 @@ public abstract class MemPointIterator implements IPointReader { protected TimeRange timeRange; protected List tsBlocks; protected boolean streamingQueryMemChunk = true; + protected QueryContext queryContext = null; - public MemPointIterator(Ordering scanOrder) { + public MemPointIterator(Ordering scanOrder, QueryContext queryContext) { this.scanOrder = scanOrder; + this.queryContext = queryContext; } public abstract TsBlock getBatch(int tsBlockIndex); @@ -72,6 +75,11 @@ public void setCurrentPageTimeRange(TimeRange timeRange) { skipToCurrentTimeRangeStartPosition(); } + /** + * counting the filtered data row is affected by: 1. Null data rows (marked as null in the bitMap) + * 2. Deleted data rows 3. Repeated timestamp rows (multiple versions may exist for the same + * timestamp) + */ protected void skipToCurrentTimeRangeStartPosition() {} protected boolean isCurrentTimeExceedTimeRange(long time) { @@ -91,6 +99,14 @@ protected void addTsBlock(TsBlock tsBlock) { tsBlocks.add(tsBlock); } + public void setQueryContext(QueryContext queryContext) { + this.queryContext = queryContext; + } + + public QueryContext getQueryContext() { + return queryContext; + } + @Override public void close() throws IOException { if (tsBlocks != null) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MemPointIteratorFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MemPointIteratorFactory.java index 20b82c338c8ef..71402b7dbe714 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MemPointIteratorFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MemPointIteratorFactory.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.utils.datastructure; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.tsfile.enums.TSDataType; @@ -34,15 +35,26 @@ private MemPointIteratorFactory() { } // TVListIterator - private static MemPointIterator single(List tvLists, int maxNumberOfPointsInPage) { + private static MemPointIterator single( + List tvLists, int maxNumberOfPointsInPage, QueryContext queryContext) { return tvLists .get(0) .iterator( - Ordering.ASC, tvLists.get(0).rowCount, null, null, null, null, maxNumberOfPointsInPage); + Ordering.ASC, + tvLists.get(0).rowCount, + null, + null, + null, + null, + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator single( - List tvLists, List deletionList, int maxNumberOfPointsInPage) { + List tvLists, + List deletionList, + int maxNumberOfPointsInPage, + QueryContext queryContext) { return tvLists .get(0) .iterator( @@ -52,7 +64,8 @@ private static MemPointIterator single( deletionList, null, null, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator single( @@ -63,7 +76,8 @@ private static MemPointIterator single( List deletionList, Integer floatPrecision, TSEncoding encoding, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return tvLists .get(0) .iterator( @@ -73,21 +87,35 @@ private static MemPointIterator single( deletionList, floatPrecision, encoding, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } // MergeSortMultiTVListIterator private static MemPointIterator mergeSort( - TSDataType tsDataType, List tvLists, int maxNumberOfPointsInPage) { + TSDataType tsDataType, + List tvLists, + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new MergeSortMultiTVListIterator( - Ordering.ASC, null, tsDataType, tvLists, null, null, null, null, maxNumberOfPointsInPage); + Ordering.ASC, + null, + tsDataType, + tvLists, + null, + null, + null, + null, + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator mergeSort( TSDataType tsDataType, List tvLists, List deletionList, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new MergeSortMultiTVListIterator( Ordering.ASC, null, @@ -97,7 +125,8 @@ private static MemPointIterator mergeSort( deletionList, null, null, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator mergeSort( @@ -109,7 +138,8 @@ private static MemPointIterator mergeSort( List deletionList, Integer floatPrecision, TSEncoding encoding, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new MergeSortMultiTVListIterator( scanOrder, globalTimeFilter, @@ -119,21 +149,35 @@ private static MemPointIterator mergeSort( deletionList, floatPrecision, encoding, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } // OrderedMultiTVListIterator private static MemPointIterator ordered( - TSDataType tsDataType, List tvLists, int maxNumberOfPointsInPage) { + TSDataType tsDataType, + List tvLists, + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new OrderedMultiTVListIterator( - Ordering.ASC, null, tsDataType, tvLists, null, null, null, null, maxNumberOfPointsInPage); + Ordering.ASC, + null, + tsDataType, + tvLists, + null, + null, + null, + null, + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator ordered( TSDataType tsDataType, List tvLists, List deletionList, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new OrderedMultiTVListIterator( Ordering.ASC, null, @@ -143,7 +187,8 @@ private static MemPointIterator ordered( deletionList, null, null, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator ordered( @@ -155,7 +200,8 @@ private static MemPointIterator ordered( List deletionList, Integer floatPrecision, TSEncoding encoding, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new OrderedMultiTVListIterator( scanOrder, globalTimeFilter, @@ -165,7 +211,8 @@ private static MemPointIterator ordered( deletionList, floatPrecision, encoding, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } // AlignedTVListIterator @@ -174,7 +221,8 @@ private static MemPointIterator single( List columnIndexList, List alignedTvLists, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return alignedTvLists .get(0) .iterator( @@ -188,7 +236,8 @@ private static MemPointIterator single( null, null, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator single( @@ -198,7 +247,8 @@ private static MemPointIterator single( List timeColumnDeletion, List> valueColumnsDeletionList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return alignedTvLists .get(0) .iterator( @@ -212,7 +262,8 @@ private static MemPointIterator single( null, null, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator single( @@ -227,7 +278,8 @@ private static MemPointIterator single( Integer floatPrecision, List encodingList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return alignedTvLists .get(0) .iterator( @@ -241,7 +293,8 @@ private static MemPointIterator single( floatPrecision, encodingList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } // MergeSortMultiAlignedTVListIterator @@ -250,7 +303,8 @@ private static MemPointIterator mergeSort( List columnIndexList, List alignedTvLists, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new MergeSortMultiAlignedTVListIterator( tsDataTypes, columnIndexList, @@ -263,7 +317,8 @@ private static MemPointIterator mergeSort( null, null, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator mergeSort( @@ -273,7 +328,8 @@ private static MemPointIterator mergeSort( List timeColumnDeletion, List> valueColumnsDeletionList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new MergeSortMultiAlignedTVListIterator( tsDataTypes, columnIndexList, @@ -286,7 +342,8 @@ private static MemPointIterator mergeSort( null, null, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator mergeSort( @@ -301,7 +358,8 @@ private static MemPointIterator mergeSort( Integer floatPrecision, List encodingList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new MergeSortMultiAlignedTVListIterator( tsDataTypes, columnIndexList, @@ -314,7 +372,8 @@ private static MemPointIterator mergeSort( floatPrecision, encodingList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } // OrderedMultiAlignedTVListIterator @@ -323,7 +382,8 @@ private static MemPointIterator ordered( List columnIndexList, List alignedTvLists, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new OrderedMultiAlignedTVListIterator( tsDataTypes, columnIndexList, @@ -336,7 +396,8 @@ private static MemPointIterator ordered( null, null, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator ordered( @@ -346,7 +407,8 @@ private static MemPointIterator ordered( List timeColumnDeletion, List> valueColumnsDeletionList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new OrderedMultiAlignedTVListIterator( tsDataTypes, columnIndexList, @@ -359,7 +421,8 @@ private static MemPointIterator ordered( null, null, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } private static MemPointIterator ordered( @@ -374,7 +437,8 @@ private static MemPointIterator ordered( Integer floatPrecision, List encodingList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new OrderedMultiAlignedTVListIterator( tsDataTypes, columnIndexList, @@ -387,31 +451,21 @@ private static MemPointIterator ordered( floatPrecision, encodingList, ignoreAllNullRows, - maxNumberOfPointsInPage); - } - - public static MemPointIterator create( - TSDataType tsDataType, List tvLists, int maxNumberOfPointsInPage) { - if (tvLists.size() == 1) { - return single(tvLists, maxNumberOfPointsInPage); - } else if (isCompleteOrdered(tvLists, null)) { - return ordered(tsDataType, tvLists, maxNumberOfPointsInPage); - } else { - return mergeSort(tsDataType, tvLists, maxNumberOfPointsInPage); - } + maxNumberOfPointsInPage, + queryContext); } public static MemPointIterator create( TSDataType tsDataType, List tvLists, - List deletionList, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { if (tvLists.size() == 1) { - return single(tvLists, deletionList, maxNumberOfPointsInPage); + return single(tvLists, maxNumberOfPointsInPage, queryContext); } else if (isCompleteOrdered(tvLists, null)) { - return ordered(tsDataType, tvLists, deletionList, maxNumberOfPointsInPage); + return ordered(tsDataType, tvLists, maxNumberOfPointsInPage, queryContext); } else { - return mergeSort(tsDataType, tvLists, deletionList, maxNumberOfPointsInPage); + return mergeSort(tsDataType, tvLists, maxNumberOfPointsInPage, queryContext); } } @@ -424,7 +478,8 @@ public static MemPointIterator create( List deletionList, Integer floatPrecision, TSEncoding encoding, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { if (tvLists.size() == 1) { return single( tvLists, @@ -434,7 +489,8 @@ public static MemPointIterator create( deletionList, floatPrecision, encoding, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } else if (isCompleteOrdered(tvLists, tvListRowCounts)) { return ordered( tsDataType, @@ -445,7 +501,8 @@ public static MemPointIterator create( deletionList, floatPrecision, encoding, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } else { return mergeSort( tsDataType, @@ -456,7 +513,8 @@ public static MemPointIterator create( deletionList, floatPrecision, encoding, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } } @@ -465,54 +523,32 @@ public static MemPointIterator create( List columnIndexList, List alignedTvLists, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { - if (alignedTvLists.size() == 1) { - return single( - tsDataTypes, columnIndexList, alignedTvLists, ignoreAllNullRows, maxNumberOfPointsInPage); - } else if (isCompleteOrdered(alignedTvLists, null)) { - return ordered( - tsDataTypes, columnIndexList, alignedTvLists, ignoreAllNullRows, maxNumberOfPointsInPage); - } else { - return mergeSort( - tsDataTypes, columnIndexList, alignedTvLists, ignoreAllNullRows, maxNumberOfPointsInPage); - } - } - - public static MemPointIterator create( - List tsDataTypes, - List columnIndexList, - List alignedTvLists, - List timeColumnDeletion, - List> valueColumnsDeletionList, - boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { if (alignedTvLists.size() == 1) { return single( tsDataTypes, columnIndexList, alignedTvLists, - timeColumnDeletion, - valueColumnsDeletionList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } else if (isCompleteOrdered(alignedTvLists, null)) { return ordered( tsDataTypes, columnIndexList, alignedTvLists, - timeColumnDeletion, - valueColumnsDeletionList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } else { return mergeSort( tsDataTypes, columnIndexList, alignedTvLists, - timeColumnDeletion, - valueColumnsDeletionList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } } @@ -528,7 +564,8 @@ public static MemPointIterator create( Integer floatPrecision, List encodingList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { if (alignedTvLists.size() == 1) { return single( tsDataTypes, @@ -542,7 +579,8 @@ public static MemPointIterator create( floatPrecision, encodingList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } else if (isCompleteOrdered(alignedTvLists, tvListRowCounts)) { return ordered( tsDataTypes, @@ -556,7 +594,8 @@ public static MemPointIterator create( floatPrecision, encodingList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } else { return mergeSort( tsDataTypes, @@ -570,7 +609,8 @@ public static MemPointIterator create( floatPrecision, encodingList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiAlignedTVListIterator.java index a10018159818b..e7e706c21de3d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiAlignedTVListIterator.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.utils.datastructure; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.tsfile.enums.TSDataType; @@ -62,7 +63,8 @@ public MergeSortMultiAlignedTVListIterator( Integer floatPrecision, List encodingList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { super( tsDataTypes, columnIndexList, @@ -75,7 +77,8 @@ public MergeSortMultiAlignedTVListIterator( floatPrecision, encodingList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); this.probeIterators = IntStream.range(0, alignedTvListIterators.size()).boxed().collect(Collectors.toSet()); this.bitMap = new BitMap(tsDataTypeList.size()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiTVListIterator.java index 8e6dd012265e8..9254a0ce1da88 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MergeSortMultiTVListIterator.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.utils.datastructure; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.tsfile.enums.TSDataType; @@ -52,7 +53,8 @@ public MergeSortMultiTVListIterator( List deletionList, Integer floatPrecision, TSEncoding encoding, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { super( scanOrder, globalTimeFilter, @@ -62,7 +64,8 @@ public MergeSortMultiTVListIterator( deletionList, floatPrecision, encoding, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); this.probeIterators = IntStream.range(0, tvListIterators.size()).boxed().collect(Collectors.toList()); this.heap = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiAlignedTVListIterator.java index 3a522f5e71462..d0d86a5b6c0d8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiAlignedTVListIterator.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.utils.datastructure; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.tsfile.block.column.ColumnBuilder; @@ -38,6 +39,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.function.LongConsumer; public abstract class MultiAlignedTVListIterator extends MemPointIterator { protected List tsDataTypeList; @@ -69,8 +71,9 @@ protected MultiAlignedTVListIterator( Integer floatPrecision, List encodingList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { - super(scanOrder); + int maxNumberOfPointsInPage, + QueryContext queryContext) { + super(scanOrder, queryContext); this.tsDataTypeList = tsDataTypeList; this.columnIndexList = columnIndexList; this.alignedTvListIterators = new ArrayList<>(alignedTvLists.size()); @@ -89,7 +92,8 @@ protected MultiAlignedTVListIterator( floatPrecision, encodingList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); alignedTvListIterators.add(iterator); } } else { @@ -107,7 +111,8 @@ protected MultiAlignedTVListIterator( floatPrecision, encodingList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); alignedTvListIterators.add(iterator); } } @@ -269,13 +274,18 @@ public TsBlock nextBatch() { } TsBlock tsBlock = builder.build(); if (pushDownFilter != null) { + LongConsumer filterRowsRecorder = + this.getQueryContext() != null && this.getQueryContext().isVerbose() + ? this.getQueryContext().getQueryStatistics()::addFilteredRowsOfRowLevel + : null; tsBlock = TsBlockUtil.applyFilterAndLimitOffsetToTsBlock( tsBlock, new TsBlockBuilder( Math.min(maxNumberOfPointsInPage, tsBlock.getPositionCount()), tsDataTypeList), pushDownFilter, - paginationController); + paginationController, + filterRowsRecorder); } else { tsBlock = paginationController.applyTsBlock(tsBlock); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiTVListIterator.java index 4b3943518399e..db35a87ed8678 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/MultiTVListIterator.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.utils.datastructure; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.tsfile.enums.TSDataType; @@ -60,8 +61,9 @@ protected MultiTVListIterator( List deletionList, Integer floatPrecision, TSEncoding encoding, - int maxNumberOfPointsInPage) { - super(scanOrder); + int maxNumberOfPointsInPage, + QueryContext queryContext) { + super(scanOrder, queryContext); this.tsDataType = tsDataType; this.tvListIterators = new ArrayList<>(tvLists.size()); if (scanOrder.isAscending()) { @@ -76,7 +78,8 @@ protected MultiTVListIterator( deletionList, null, null, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); tvListIterators.add(iterator); } } else { @@ -91,7 +94,8 @@ protected MultiTVListIterator( deletionList, null, null, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); tvListIterators.add(iterator); } } @@ -143,6 +147,7 @@ public boolean hasNextBatch() { @Override public TsBlock nextBatch() { TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(tsDataType)); + long filteredRowsByPushDownFilter = 0; switch (tsDataType) { case BOOLEAN: while (hasNextTimeValuePair() && builder.getPositionCount() < maxNumberOfPointsInPage) { @@ -152,6 +157,8 @@ public TsBlock nextBatch() { builder.getTimeColumnBuilder().writeLong(currentTime); builder.getColumnBuilder(0).writeBoolean(aBoolean); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } next(); } @@ -165,6 +172,8 @@ public TsBlock nextBatch() { builder.getTimeColumnBuilder().writeLong(currentTime); builder.getColumnBuilder(0).writeInt(anInt); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } next(); } @@ -178,6 +187,8 @@ public TsBlock nextBatch() { builder.getTimeColumnBuilder().writeLong(currentTime); builder.getColumnBuilder(0).writeLong(aLong); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } next(); } @@ -195,6 +206,8 @@ public TsBlock nextBatch() { builder.getTimeColumnBuilder().writeLong(currentTime); builder.getColumnBuilder(0).writeFloat(aFloat); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } next(); } @@ -212,6 +225,8 @@ public TsBlock nextBatch() { builder.getTimeColumnBuilder().writeLong(currentTime); builder.getColumnBuilder(0).writeDouble(aDouble); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } next(); } @@ -227,6 +242,8 @@ public TsBlock nextBatch() { builder.getTimeColumnBuilder().writeLong(currentTime); builder.getColumnBuilder(0).writeBinary(binary); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } next(); } @@ -235,6 +252,13 @@ public TsBlock nextBatch() { throw new UnSupportedDataTypeException( String.format("Data type %s is not supported.", tsDataType)); } + + if (this.getQueryContext().isVerbose() && filteredRowsByPushDownFilter > 0) { + this.getQueryContext() + .getQueryStatistics() + .addFilteredRowsOfRowLevel(filteredRowsByPushDownFilter); + } + // There is no need to process pushDownFilter here because it has been applied when // constructing the tsBlock TsBlock tsBlock = paginationController.applyTsBlock(builder.build()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/OrderedMultiAlignedTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/OrderedMultiAlignedTVListIterator.java index e5b2cdab58bf7..e6b5afb6ec396 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/OrderedMultiAlignedTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/OrderedMultiAlignedTVListIterator.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.utils.datastructure; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.tsfile.enums.TSDataType; @@ -49,7 +50,8 @@ public OrderedMultiAlignedTVListIterator( Integer floatPrecision, List encodingList, boolean ignoreAllNullRows, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { super( tsDataTypes, columnIndexList, @@ -62,7 +64,8 @@ public OrderedMultiAlignedTVListIterator( floatPrecision, encodingList, ignoreAllNullRows, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); this.bitMap = new BitMap(tsDataTypeList.size()); this.ignoreAllNullRows = ignoreAllNullRows; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/OrderedMultiTVListIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/OrderedMultiTVListIterator.java index 27f79eca846d7..2b4eab8a7184a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/OrderedMultiTVListIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/OrderedMultiTVListIterator.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.utils.datastructure; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.tsfile.enums.TSDataType; @@ -39,7 +40,8 @@ public OrderedMultiTVListIterator( List deletionList, Integer floatPrecision, TSEncoding encoding, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext context) { super( scanOrder, globalTimeFilter, @@ -49,7 +51,8 @@ public OrderedMultiTVListIterator( deletionList, floatPrecision, encoding, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + context); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index b3bedd62e8fdf..8b571fed01f8a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -787,7 +787,8 @@ public TVListIterator iterator( List deletionList, Integer floatPrecision, TSEncoding encoding, - int maxNumberOfPointsInPage) { + int maxNumberOfPointsInPage, + QueryContext queryContext) { return new TVListIterator( scanOrder, rowCount, @@ -795,7 +796,8 @@ public TVListIterator iterator( deletionList, floatPrecision, encoding, - maxNumberOfPointsInPage); + maxNumberOfPointsInPage, + queryContext); } /* TVList Iterator */ @@ -820,8 +822,9 @@ public TVListIterator( List deletionList, Integer floatPrecision, TSEncoding encoding, - int maxNumberOfPointsInPage) { - super(scanOrder); + int maxNumberOfPointsInPage, + QueryContext queryContext) { + super(scanOrder, queryContext); this.globalTimeFilter = globalTimeFilter; this.deletionList = deletionList; this.floatPrecision = floatPrecision != null ? floatPrecision : 0; @@ -854,6 +857,10 @@ protected void skipToCurrentTimeRangeStartPosition() { } if (searchTimestamp > outer.getMaxTime()) { // all satisfied data has been consumed + // the data with same timestamp could exist multi times, which means filter multi times + if (this.getQueryContext() != null && this.getQueryContext().isVerbose()) { + this.getQueryContext().getQueryStatistics().addFilteredRowsOfRowLevel(rows - index); + } index = rows; probeNext = true; return; @@ -871,6 +878,9 @@ protected void skipToCurrentTimeRangeStartPosition() { } if (searchTimestamp < outer.getMinTime()) { // all satisfied data has been consumed + if (this.getQueryContext() != null && this.getQueryContext().isVerbose()) { + this.getQueryContext().getQueryStatistics().addFilteredRowsOfRowLevel(rows - index); + } index = rows; probeNext = true; return; @@ -884,6 +894,10 @@ protected void skipToCurrentTimeRangeStartPosition() { } int newIndex = getScanOrderIndex(indexInTVList); if (newIndex > index) { + // count the data rows skipped by binary search + if (this.getQueryContext() != null && this.getQueryContext().isVerbose()) { + this.getQueryContext().getQueryStatistics().addFilteredRowsOfRowLevel(newIndex - index); + } index = newIndex; } @@ -932,17 +946,31 @@ && getTime(getScanOrderIndex(index - 1)) == getTime(getScanOrderIndex(index))) { probeNext = true; } + /** + * The data row that is not deleted and not satisfied by timeFilter has the qualification to be + * recorded + */ protected void skipDeletedOrTimeNotSatisfiedRows() { + long filteredRows = 0; while (index < rows) { if (!isNullValue(getValueIndex(getScanOrderIndex(index)))) { long time = getTime(getScanOrderIndex(index)); - if (!isPointDeleted(time, deletionList, deleteCursor, scanOrder) - && isTimeSatisfied(time)) { - break; + if (isPointDeleted(time, deletionList, deleteCursor, scanOrder)) { + index++; + continue; + } + if (!isTimeSatisfied(time)) { + filteredRows++; + index++; + continue; } + break; } index++; } + if (this.getQueryContext() != null && this.getQueryContext().isVerbose()) { + this.getQueryContext().getQueryStatistics().addFilteredRowsOfRowLevel(filteredRows); + } } protected boolean isTimeSatisfied(long timestamp) { @@ -1007,6 +1035,13 @@ public TsBlock nextBatch() { Math.min(maxNumberOfPointsInPage, rows - index)); TsBlockBuilder builder = new TsBlockBuilder(maxRowCountOfCurrentBatch, Collections.singletonList(dataType)); + + long[] filteredRowsByTimeFilter = + this.getQueryContext() != null && this.getQueryContext().isVerbose() + ? new long[] {0} + : null; + long filteredRowsByPushDownFilter = 0; + switch (dataType) { case BOOLEAN: while (index < rows @@ -1016,10 +1051,8 @@ public TsBlock nextBatch() { if (isCurrentTimeExceedTimeRange(time)) { break; } - if (!isNullValue(getValueIndex(getScanOrderIndex(index))) - && !isPointDeleted(time, deletionList, deleteCursor, scanOrder) - && isLatestPoint(index, time) - && isTimeSatisfied(time)) { + if (!isInvalidRow( + time, index, deletionList, deleteCursor, scanOrder, filteredRowsByTimeFilter)) { boolean aBoolean = getBoolean(getScanOrderIndex(index)); if (pushDownFilter == null || pushDownFilter.satisfyBoolean(time, aBoolean)) { if (paginationController.hasCurOffset()) { @@ -1031,6 +1064,8 @@ && isTimeSatisfied(time)) { builder.getTimeColumnBuilder().writeLong(time); builder.getColumnBuilder(0).writeBoolean(aBoolean); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } } index++; @@ -1045,10 +1080,8 @@ && isTimeSatisfied(time)) { if (isCurrentTimeExceedTimeRange(time)) { break; } - if (!isNullValue(getValueIndex(getScanOrderIndex(index))) - && !isPointDeleted(time, deletionList, deleteCursor, scanOrder) - && isLatestPoint(index, time) - && isTimeSatisfied(time)) { + if (!isInvalidRow( + time, index, deletionList, deleteCursor, scanOrder, filteredRowsByTimeFilter)) { int anInt = getInt(getScanOrderIndex(index)); if (pushDownFilter == null || pushDownFilter.satisfyInteger(time, anInt)) { if (paginationController.hasCurOffset()) { @@ -1060,6 +1093,8 @@ && isTimeSatisfied(time)) { builder.getTimeColumnBuilder().writeLong(time); builder.getColumnBuilder(0).writeInt(anInt); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } } index++; @@ -1074,10 +1109,8 @@ && isTimeSatisfied(time)) { if (isCurrentTimeExceedTimeRange(time)) { break; } - if (!isNullValue(getValueIndex(getScanOrderIndex(index))) - && !isPointDeleted(time, deletionList, deleteCursor, scanOrder) - && isLatestPoint(index, time) - && isTimeSatisfied(time)) { + if (!isInvalidRow( + time, index, deletionList, deleteCursor, scanOrder, filteredRowsByTimeFilter)) { long aLong = getLong(getScanOrderIndex(index)); if (pushDownFilter == null || pushDownFilter.satisfyLong(time, aLong)) { if (paginationController.hasCurOffset()) { @@ -1089,6 +1122,8 @@ && isTimeSatisfied(time)) { builder.getTimeColumnBuilder().writeLong(time); builder.getColumnBuilder(0).writeLong(aLong); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } } index++; @@ -1102,10 +1137,8 @@ && isTimeSatisfied(time)) { if (isCurrentTimeExceedTimeRange(time)) { break; } - if (!isNullValue(getValueIndex(getScanOrderIndex(index))) - && !isPointDeleted(time, deletionList, deleteCursor, scanOrder) - && isLatestPoint(index, time) - && isTimeSatisfied(time)) { + if (!isInvalidRow( + time, index, deletionList, deleteCursor, scanOrder, filteredRowsByTimeFilter)) { float aFloat = roundValueWithGivenPrecision( getFloat(getScanOrderIndex(index)), floatPrecision, encoding); @@ -1119,6 +1152,8 @@ && isTimeSatisfied(time)) { builder.getTimeColumnBuilder().writeLong(time); builder.getColumnBuilder(0).writeFloat(aFloat); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } } index++; @@ -1132,10 +1167,8 @@ && isTimeSatisfied(time)) { if (isCurrentTimeExceedTimeRange(time)) { break; } - if (!isNullValue(getValueIndex(getScanOrderIndex(index))) - && !isPointDeleted(time, deletionList, deleteCursor, scanOrder) - && isLatestPoint(index, time) - && isTimeSatisfied(time)) { + if (!isInvalidRow( + time, index, deletionList, deleteCursor, scanOrder, filteredRowsByTimeFilter)) { double aDouble = roundValueWithGivenPrecision( getDouble(getScanOrderIndex(index)), floatPrecision, encoding); @@ -1149,6 +1182,8 @@ && isTimeSatisfied(time)) { builder.getTimeColumnBuilder().writeLong(time); builder.getColumnBuilder(0).writeDouble(aDouble); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } } index++; @@ -1165,10 +1200,8 @@ && isTimeSatisfied(time)) { if (isCurrentTimeExceedTimeRange(time)) { break; } - if (!isNullValue(getValueIndex(getScanOrderIndex(index))) - && !isPointDeleted(time, deletionList, deleteCursor, scanOrder) - && isLatestPoint(index, time) - && isTimeSatisfied(time)) { + if (!isInvalidRow( + time, index, deletionList, deleteCursor, scanOrder, filteredRowsByTimeFilter)) { Binary binary = getBinary(getScanOrderIndex(index)); if (pushDownFilter == null || pushDownFilter.satisfyBinary(time, binary)) { if (paginationController.hasCurOffset()) { @@ -1180,6 +1213,8 @@ && isTimeSatisfied(time)) { builder.getTimeColumnBuilder().writeLong(time); builder.getColumnBuilder(0).writeBinary(binary); builder.declarePosition(); + } else { + filteredRowsByPushDownFilter++; } } index++; @@ -1189,6 +1224,21 @@ && isTimeSatisfied(time)) { throw new UnSupportedDataTypeException( String.format("Data type %s is not supported.", dataType)); } + + // count the filtered row from time filter and other filter + if (this.getQueryContext() != null && this.getQueryContext().isVerbose()) { + if (filteredRowsByTimeFilter != null && filteredRowsByTimeFilter[0] > 0) { + this.getQueryContext() + .getQueryStatistics() + .addFilteredRowsOfRowLevel(filteredRowsByTimeFilter[0]); + } + if (filteredRowsByPushDownFilter > 0) { + this.getQueryContext() + .getQueryStatistics() + .addFilteredRowsOfRowLevel(filteredRowsByPushDownFilter); + } + } + // There is no need to process pushDownFilter and paginationController here because it has // been applied when constructing the tsBlock TsBlock tsBlock = builder.build(); @@ -1196,6 +1246,28 @@ && isTimeSatisfied(time)) { return tsBlock; } + private boolean isInvalidRow( + long time, + int index, + List deletionList, + int[] deleteCursor, + Ordering scanOrder, + long[] filteredRowsByTimeFilter) { + + if (isNullValue(getValueIndex(getScanOrderIndex(index))) + || isPointDeleted(time, deletionList, deleteCursor, scanOrder) + || !isLatestPoint(index, time)) { + return true; + } + if (!isTimeSatisfied(time)) { + if (filteredRowsByTimeFilter != null) { + filteredRowsByTimeFilter[0]++; + } + return true; + } + return false; + } + protected boolean isLatestPoint(int rowIndex, long currentTime) { if (scanOrder.isAscending()) { return rowIndex == rows - 1 || currentTime != getTime(getScanOrderIndex(rowIndex + 1)); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/ExchangeOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/ExchangeOperatorTest.java new file mode 100644 index 0000000000000..a5f30cf6d611b --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/ExchangeOperatorTest.java @@ -0,0 +1,161 @@ +/* + * 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.iotdb.db.queryengine.execution.operator; + +import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; +import org.apache.iotdb.db.queryengine.common.FragmentInstanceId; +import org.apache.iotdb.db.queryengine.common.PlanFragmentId; +import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.execution.driver.DriverContext; +import org.apache.iotdb.db.queryengine.execution.exchange.source.ISourceHandle; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceStateMachine; +import org.apache.iotdb.db.queryengine.execution.operator.source.ExchangeOperator; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.Collections; +import java.util.concurrent.ExecutorService; + +import static org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext.createFragmentInstanceContext; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class ExchangeOperatorTest { + + private ExecutorService instanceNotificationExecutor; + + @Before + public void setUp() { + instanceNotificationExecutor = + IoTDBThreadPoolFactory.newFixedThreadPool(1, "test-exchange-node-notification"); + } + + @After + public void tearDown() { + instanceNotificationExecutor.shutdown(); + } + + private OperatorContext createOperatorContext() { + QueryId queryId = new QueryId("test_exchange_query"); + FragmentInstanceId instanceId = + new FragmentInstanceId(new PlanFragmentId(queryId, 0), "test-instance"); + FragmentInstanceStateMachine stateMachine = + new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor); + FragmentInstanceContext fragmentInstanceContext = + createFragmentInstanceContext(instanceId, stateMachine); + DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0); + driverContext.addOperatorContext( + 1, new PlanNodeId("1"), ExchangeOperator.class.getSimpleName()); + return driverContext.getOperatorContexts().get(0); + } + + /** + * When sourceHandle.receive() returns a non-null TsBlock, receivedSizeInBytes should increase. + */ + @Test + public void testNextAccumulatesReceivedSizeWhenBlockIsNotNull() throws Exception { + ISourceHandle sourceHandle = Mockito.mock(ISourceHandle.class); + TsBlock tsBlock = buildTsBlock(10); + Mockito.when(sourceHandle.receive()).thenReturn(tsBlock); + + try (ExchangeOperator operator = + new ExchangeOperator(createOperatorContext(), sourceHandle, new PlanNodeId("source1")); ) { + + TsBlock result = operator.next(); + assertEquals(tsBlock, result); + long expectedSize = tsBlock.getSizeInBytes(); + assertEquals( + expectedSize, + ((java.util.concurrent.atomic.AtomicLong) + operator + .getOperatorContext() + .getSpecifiedInfo() + .get(ExchangeOperator.SIZE_IN_BYTES)) + .get()); + } + } + + /** When sourceHandle.receive() returns null, receivedSizeInBytes should remain 0. */ + @Test + public void testNextDoesNotAccumulateWhenBlockIsNull() throws Exception { + ISourceHandle sourceHandle = Mockito.mock(ISourceHandle.class); + Mockito.when(sourceHandle.receive()).thenReturn(null); + + try (ExchangeOperator operator = + new ExchangeOperator(createOperatorContext(), sourceHandle, new PlanNodeId("source2"))) { + + TsBlock result = operator.next(); + + assertNull(result); + assertEquals( + 0L, + ((java.util.concurrent.atomic.AtomicLong) + operator + .getOperatorContext() + .getSpecifiedInfo() + .get(ExchangeOperator.SIZE_IN_BYTES)) + .get()); + } + } + + /** Multiple calls to next() should accumulate sizes correctly. */ + @Test + public void testNextAccumulatesAcrossMultipleCalls() throws Exception { + ISourceHandle sourceHandle = Mockito.mock(ISourceHandle.class); + TsBlock block1 = buildTsBlock(5); + TsBlock block2 = buildTsBlock(8); + Mockito.when(sourceHandle.receive()).thenReturn(block1).thenReturn(null).thenReturn(block2); + + ExchangeOperator operator = + new ExchangeOperator(createOperatorContext(), sourceHandle, new PlanNodeId("source3")); + + operator.next(); // block1 — should accumulate + operator.next(); // null — should not accumulate + operator.next(); // block2 — should accumulate + + long expectedSize = block1.getSizeInBytes() + block2.getSizeInBytes(); + assertEquals( + expectedSize, + ((java.util.concurrent.atomic.AtomicLong) + operator + .getOperatorContext() + .getSpecifiedInfo() + .get(ExchangeOperator.SIZE_IN_BYTES)) + .get()); + } + + private TsBlock buildTsBlock(int rowCount) { + TsBlockBuilder builder = new TsBlockBuilder(Collections.singletonList(TSDataType.INT32)); + for (int i = 0; i < rowCount; i++) { + builder.getTimeColumnBuilder().writeLong(i); + builder.getColumnBuilder(0).writeInt(i); + builder.declarePosition(); + } + return builder.build(); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorMemoryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorMemoryTest.java index de342504e0a2c..ae1b9458f0311 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorMemoryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorMemoryTest.java @@ -227,7 +227,9 @@ public void alignedSeriesScanOperatorTest() { @Test public void exchangeOperatorTest() { - ExchangeOperator exchangeOperator = new ExchangeOperator(null, null, null); + OperatorContext operatorContext = Mockito.mock(OperatorContext.class); + Mockito.when(operatorContext.getSpecifiedInfo()).thenReturn(new java.util.HashMap<>()); + ExchangeOperator exchangeOperator = new ExchangeOperator(operatorContext, null, null); assertEquals(DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES, exchangeOperator.calculateMaxPeekMemory()); assertEquals(DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES, exchangeOperator.calculateMaxReturnSize()); @@ -241,8 +243,10 @@ public void pipelineExchangeOperatorTest() { Mockito.when(child.calculateMaxReturnSize()).thenReturn(1024L); Mockito.when(child.calculateRetainedSizeAfterCallingNext()).thenReturn(512L); + OperatorContext operatorContext = Mockito.mock(OperatorContext.class); + Mockito.when(operatorContext.getSpecifiedInfo()).thenReturn(new java.util.HashMap<>()); ExchangeOperator exchangeOperator = - new ExchangeOperator(null, null, null, child.calculateMaxReturnSize()); + new ExchangeOperator(operatorContext, null, null, child.calculateMaxReturnSize()); assertEquals(1024L, exchangeOperator.calculateMaxPeekMemory()); assertEquals(1024L, exchangeOperator.calculateMaxReturnSize()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/sink/SinkOperatorDownStreamNodeIdTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/sink/SinkOperatorDownStreamNodeIdTest.java new file mode 100644 index 0000000000000..12e295d191aba --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/sink/SinkOperatorDownStreamNodeIdTest.java @@ -0,0 +1,173 @@ +/* + * 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.iotdb.db.queryengine.execution.operator.sink; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; +import org.apache.iotdb.db.queryengine.common.FragmentInstanceId; +import org.apache.iotdb.db.queryengine.common.PlanFragmentId; +import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.execution.exchange.sink.DownStreamChannelLocation; +import org.apache.iotdb.db.queryengine.execution.fragment.DataNodeQueryContext; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceStateMachine; +import org.apache.iotdb.db.queryengine.execution.operator.Operator; +import org.apache.iotdb.db.queryengine.plan.analyze.TypeProvider; +import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanContext; +import org.apache.iotdb.db.queryengine.plan.planner.OperatorTreeGenerator; +import org.apache.iotdb.db.queryengine.plan.planner.TableOperatorGenerator; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.sink.IdentitySinkNode; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableMetadataImpl; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceId; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutorService; + +import static org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext.createFragmentInstanceContext; +import static org.apache.iotdb.db.queryengine.execution.operator.sink.IdentitySinkOperator.DOWNSTREAM_PLAN_NODE_ID; +import static org.junit.Assert.assertEquals; + +public class SinkOperatorDownStreamNodeIdTest { + + private static ExecutorService instanceNotificationExecutor; + public final Metadata metadata = new TableMetadataImpl(); + + @BeforeClass + public static void setUp() { + instanceNotificationExecutor = + IoTDBThreadPoolFactory.newFixedThreadPool(1, "sink-downstream-id-notification"); + } + + @AfterClass + public static void tearDown() { + instanceNotificationExecutor.shutdown(); + } + + private LocalExecutionPlanContext createLocalExecutionPlanContext( + TypeProvider typeProvider, String query_name, int queryId) { + + QueryId PlanFragmenQueryId = new QueryId(query_name); + FragmentInstanceId instanceId = + new FragmentInstanceId( + new PlanFragmentId(PlanFragmenQueryId, queryId), "stub-instance" + queryId); + FragmentInstanceStateMachine stateMachine = + new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor); + DataRegion dataRegion = Mockito.mock(DataRegion.class); + FragmentInstanceContext fragmentInstanceContext = + createFragmentInstanceContext(instanceId, stateMachine); + fragmentInstanceContext.setDataRegion(dataRegion); + + return new LocalExecutionPlanContext( + typeProvider, fragmentInstanceContext, new DataNodeQueryContext(1)); + } + + // for table model + @Test + public void testSingleDownStreamNodeIdInTable() throws Exception { + + PlanNodeId testIdentitySinkNode = new PlanNodeId("testIdentitySinkNode"); + PlanNode mockedPlanNode = Mockito.mock(PlanNode.class); + LocalExecutionPlanContext context = + createLocalExecutionPlanContext( + new TypeProvider(), "query_name_sink_operator_downstream_id_1", 10001); + TableOperatorGenerator tableOperatorGenerator = new TableOperatorGenerator(metadata); + Mockito.when(mockedPlanNode.accept(tableOperatorGenerator, context)) + .thenReturn(Mockito.mock(Operator.class)); + + String remotePlanNodeId = "178"; + DownStreamChannelLocation downStreamChannelLocation = + new DownStreamChannelLocation( + new TEndPoint("test", 1), + new TFragmentInstanceId("query_name_sink_operator_downstream_id_1", 10001, "test"), + remotePlanNodeId); + try (IdentitySinkNode identitySinkNode = + new IdentitySinkNode( + testIdentitySinkNode, + Collections.singletonList(mockedPlanNode), + Collections.singletonList(downStreamChannelLocation))) { + + Operator identitySinkOperatorOfTable = + identitySinkNode.accept(tableOperatorGenerator, context); + assertEquals( + identitySinkOperatorOfTable + .getOperatorContext() + .getSpecifiedInfo() + .get(DOWNSTREAM_PLAN_NODE_ID) + .toString(), + remotePlanNodeId); + } + } + + // for tree model + @Test + public void testSingleDownStreamNodeIdInTree() throws Exception { + + PlanNodeId testIdentitySinkNode = new PlanNodeId("testIdentitySinkNode"); + PlanNode mockedChildPlanNode = Mockito.mock(PlanNode.class); + LocalExecutionPlanContext context = + createLocalExecutionPlanContext( + new TypeProvider(), "query_name_sink_operator_downstream_id_2", 10002); + + OperatorTreeGenerator operatorTreeGenerator = Mockito.spy(new OperatorTreeGenerator()); + + List dummyChildrenOperator = Collections.singletonList(Mockito.mock(Operator.class)); + Mockito.doReturn(dummyChildrenOperator) + .when(operatorTreeGenerator) + .dealWithConsumeChildrenOneByOneNode( + Mockito.any(PlanNode.class), Mockito.any(LocalExecutionPlanContext.class)); + + Mockito.when(mockedChildPlanNode.accept(operatorTreeGenerator, context)) + .thenReturn(Mockito.mock(Operator.class)); + + String remotePlanNodeId = "123"; + DownStreamChannelLocation downStreamChannelLocation = + new DownStreamChannelLocation( + new TEndPoint("test", 1), + new TFragmentInstanceId("query_name_sink_operator_downstream_id_2", 10002, "test"), + remotePlanNodeId); + + try (IdentitySinkNode identitySinkNode = + new IdentitySinkNode( + testIdentitySinkNode, + Collections.singletonList(mockedChildPlanNode), + Collections.singletonList(downStreamChannelLocation))) { + + Operator identitySinkOperatorOfTree = identitySinkNode.accept(operatorTreeGenerator, context); + + assertEquals( + identitySinkOperatorOfTree + .getOperatorContext() + .getSpecifiedInfo() + .get(DOWNSTREAM_PLAN_NODE_ID) + .toString(), + remotePlanNodeId); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/FragmentInstanceSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/FragmentInstanceSerdeTest.java index 3596ea550c774..6f488dcafcda5 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/FragmentInstanceSerdeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/FragmentInstanceSerdeTest.java @@ -77,6 +77,7 @@ public void testSerializeAndDeserializeForTree1() throws IllegalPathException { QueryType.READ, config.getQueryTimeoutThreshold(), sessionInfo, + false, false); // test FI with StorageExecutor TRegionReplicaSet regionReplicaSet = @@ -120,6 +121,7 @@ public void testSerializeAndDeserializeWithNullFilter() throws IllegalPathExcept QueryType.READ, config.getQueryTimeoutThreshold(), sessionInfo, + false, false); TRegionReplicaSet regionReplicaSet = new TRegionReplicaSet( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java index 3ea888057ccb7..431610b582b0e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteMaterializerTest.java @@ -117,7 +117,8 @@ public static void prepareEnv() { Mockito.any(), // ExplainType Mockito.anyLong(), // timeOut Mockito.anyBoolean(), // userQuery - Mockito.anyBoolean())) // debug + Mockito.anyBoolean(), // debug + Mockito.anyBoolean())) // isVerbose .thenReturn(mockResult); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java index 25b00fd66acdf..e4b9bf5edc3a4 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CteSubqueryTest.java @@ -140,7 +140,8 @@ private void mockExecuteForTableModel() throws IoTDBException { Mockito.any(), // ExplainType Mockito.anyLong(), // timeOut Mockito.anyBoolean(), // userQuery - Mockito.anyBoolean())) // debug + Mockito.anyBoolean(), // debug + Mockito.anyBoolean())) // isVerbose .thenReturn(mockResult); // Create QueryExecution mock diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/MemChunkDeserializeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/MemChunkDeserializeTest.java index e043da9125134..e2319406a8a95 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/MemChunkDeserializeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/MemChunkDeserializeTest.java @@ -301,7 +301,7 @@ private ReadOnlyMemChunk getReadOnlyChunk(WritableMemChunk memChunk, TSDataType memTableMap.put(deviceID, memChunkGroup); IMemTable memTable = new PrimitiveMemTable(storageGroup, dataRegionId, memTableMap); - QueryContext context = new QueryContext(false); + QueryContext context = new QueryContext(false, false); NonAlignedFullPath nonAlignedFullPath = new NonAlignedFullPath( deviceID, @@ -325,7 +325,7 @@ private ReadOnlyMemChunk getAlignedReadOnlyChunk( memTableMap.put(deviceID, memChunkGroup); IMemTable memTable = new PrimitiveMemTable(storageGroup, dataRegionId, memTableMap); - QueryContext context = new QueryContext(false); + QueryContext context = new QueryContext(false, false); AlignedFullPath alignedFullPath = new AlignedFullPath(deviceID, measurementList, schemaList); return memTable.query(context, alignedFullPath, Long.MIN_VALUE, null, null); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java index 169dd11042fa0..7fb46b14cf2b2 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java @@ -142,7 +142,13 @@ public void memSeriesSortIteratorTest() throws IOException, QueryProcessExceptio tvListQueryMap.put(series.getWorkingTVList(), series.getWorkingTVList().rowCount()); ReadOnlyMemChunk readableChunk = new ReadOnlyMemChunk( - new QueryContext(false), "s1", dataType, TSEncoding.PLAIN, tvListQueryMap, null, null); + new QueryContext(false, false), + "s1", + dataType, + TSEncoding.PLAIN, + tvListQueryMap, + null, + null); IPointReader it = readableChunk.getPointReader(); int i = 0; while (it.hasNextTimeValuePair()) { @@ -307,7 +313,8 @@ public void simpleTest() throws IOException, QueryProcessException, MetadataExce } ReadOnlyMemChunk memChunk = - memTable.query(new QueryContext(false), nonAlignedFullPath, Long.MIN_VALUE, null, null); + memTable.query( + new QueryContext(false, false), nonAlignedFullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); for (int i = 0; i < dataSize; i++) { iterator.hasNextTimeValuePair(); @@ -403,7 +410,11 @@ public void queryWithDeletionTest() throws IOException, QueryProcessException, M modsToMemtable.add(new Pair<>(deletion, memTable)); ReadOnlyMemChunk memChunk = memTable.query( - new QueryContext(false), nonAlignedFullPath, Long.MIN_VALUE, modsToMemtable, null); + new QueryContext(false, false), + nonAlignedFullPath, + Long.MIN_VALUE, + modsToMemtable, + null); IPointReader iterator = memChunk.getPointReader(); int cnt = 0; while (iterator.hasNextTimeValuePair()) { @@ -449,7 +460,7 @@ public void queryAlignChuckWithDeletionTest() modsToMemtable.add(new Pair<>(deletion, memTable)); ReadOnlyMemChunk memChunk = memTable.query( - new QueryContext(false), alignedFullPath, Long.MIN_VALUE, modsToMemtable, null); + new QueryContext(false, false), alignedFullPath, Long.MIN_VALUE, modsToMemtable, null); IPointReader iterator = memChunk.getPointReader(); int cnt = 0; while (iterator.hasNextTimeValuePair()) { @@ -489,7 +500,7 @@ private void write( Collections.emptyMap())); IPointReader tvPair = memTable - .query(new QueryContext(false), fullPath, Long.MIN_VALUE, null, null) + .query(new QueryContext(false, false), fullPath, Long.MIN_VALUE, null, null) .getPointReader(); Arrays.sort(ret); TimeValuePair last = null; @@ -539,7 +550,7 @@ private void writeVector(IMemTable memTable) Collections.emptyMap()))); IPointReader tvPair = memTable - .query(new QueryContext(false), tmpAlignedFullPath, Long.MIN_VALUE, null, null) + .query(new QueryContext(false, false), tmpAlignedFullPath, Long.MIN_VALUE, null, null) .getPointReader(); for (int i = 0; i < 100; i++) { tvPair.hasNextTimeValuePair(); @@ -568,7 +579,7 @@ private void writeVector(IMemTable memTable) tvPair = memTable - .query(new QueryContext(false), tmpAlignedFullPath, Long.MIN_VALUE, null, null) + .query(new QueryContext(false, false), tmpAlignedFullPath, Long.MIN_VALUE, null, null) .getPointReader(); for (int i = 0; i < 100; i++) { tvPair.hasNextTimeValuePair(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java index 72406a2e4a74b..897bdc86a9fe9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemAlignedChunkLoaderTest.java @@ -63,7 +63,7 @@ public class MemAlignedChunkLoaderTest { public void testMemAlignedChunkLoader() throws IOException { AlignedReadOnlyMemChunk chunk = Mockito.mock(AlignedReadOnlyMemChunk.class); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); - QueryContext ctx = new QueryContext(false); + QueryContext ctx = new QueryContext(false, false); MemAlignedChunkLoader memAlignedChunkLoader = new MemAlignedChunkLoader(ctx, chunk); try { @@ -116,7 +116,7 @@ public void testMemAlignedChunkLoader() throws IOException { alignedTvListMap.keySet().stream().map(x -> (AlignedTVList) x).collect(Collectors.toList()); MemPointIterator timeValuePairIterator = MemPointIteratorFactory.create( - dataTypes, null, alignedTvLists, false, maxNumberOfPointsInPage); + dataTypes, null, alignedTvLists, false, maxNumberOfPointsInPage, null); timeValuePairIterator.setStreamingQueryMemChunk(false); timeValuePairIterator.nextBatch(); Mockito.when(chunk.getMemPointIterator()).thenReturn(timeValuePairIterator); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java index 986b88ec11269..885c329b86dc2 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/MemChunkLoaderTest.java @@ -68,14 +68,16 @@ public void testBooleanMemChunkLoader() throws IOException { Map booleanTvListMap = buildBooleanTvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(booleanTvListMap); List booleanTvLists = new ArrayList<>(booleanTvListMap.keySet()); + QueryContext queryContext = new QueryContext(false, false); MemPointIterator timeValuePairIterator = - MemPointIteratorFactory.create(TSDataType.BOOLEAN, booleanTvLists, maxNumberOfPointsInPage); + MemPointIteratorFactory.create( + TSDataType.BOOLEAN, booleanTvLists, maxNumberOfPointsInPage, queryContext); timeValuePairIterator.setStreamingQueryMemChunk(false); timeValuePairIterator.nextBatch(); Mockito.when(chunk.getMemPointIterator()).thenReturn(timeValuePairIterator); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); - MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(false), chunk); + MemChunkLoader memChunkLoader = new MemChunkLoader(queryContext, chunk); try { memChunkLoader.loadChunk(chunkMetadata); fail(); @@ -148,14 +150,16 @@ public void testInt32MemChunkLoader() throws IOException { Map int32TvListMap = buildInt32TvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(int32TvListMap); List int32TvLists = new ArrayList<>(int32TvListMap.keySet()); + QueryContext queryContext = new QueryContext(false, false); MemPointIterator timeValuePairIterator = - MemPointIteratorFactory.create(TSDataType.INT32, int32TvLists, maxNumberOfPointsInPage); + MemPointIteratorFactory.create( + TSDataType.INT32, int32TvLists, maxNumberOfPointsInPage, queryContext); timeValuePairIterator.setStreamingQueryMemChunk(false); timeValuePairIterator.nextBatch(); Mockito.when(chunk.getMemPointIterator()).thenReturn(timeValuePairIterator); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); - MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(false), chunk); + MemChunkLoader memChunkLoader = new MemChunkLoader(queryContext, chunk); try { memChunkLoader.loadChunk(chunkMetadata); fail(); @@ -228,14 +232,16 @@ public void testInt64MemChunkLoader() throws IOException { Map int64TvListMap = buildInt64TvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(int64TvListMap); List int64TvLists = new ArrayList<>(int64TvListMap.keySet()); + QueryContext queryContext = new QueryContext(false, false); MemPointIterator timeValuePairIterator = - MemPointIteratorFactory.create(TSDataType.INT64, int64TvLists, maxNumberOfPointsInPage); + MemPointIteratorFactory.create( + TSDataType.INT64, int64TvLists, maxNumberOfPointsInPage, queryContext); timeValuePairIterator.setStreamingQueryMemChunk(false); timeValuePairIterator.nextBatch(); Mockito.when(chunk.getMemPointIterator()).thenReturn(timeValuePairIterator); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); - MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(false), chunk); + MemChunkLoader memChunkLoader = new MemChunkLoader(queryContext, chunk); try { memChunkLoader.loadChunk(chunkMetadata); fail(); @@ -308,14 +314,16 @@ public void testFloatMemChunkLoader() throws IOException { Map floatTvListMap = buildFloatTvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(floatTvListMap); List floatTvLists = new ArrayList<>(floatTvListMap.keySet()); + QueryContext queryContext = new QueryContext(false, false); MemPointIterator timeValuePairIterator = - MemPointIteratorFactory.create(TSDataType.FLOAT, floatTvLists, maxNumberOfPointsInPage); + MemPointIteratorFactory.create( + TSDataType.FLOAT, floatTvLists, maxNumberOfPointsInPage, queryContext); timeValuePairIterator.setStreamingQueryMemChunk(false); timeValuePairIterator.nextBatch(); Mockito.when(chunk.getMemPointIterator()).thenReturn(timeValuePairIterator); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); - MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(false), chunk); + MemChunkLoader memChunkLoader = new MemChunkLoader(queryContext, chunk); try { memChunkLoader.loadChunk(chunkMetadata); fail(); @@ -388,14 +396,16 @@ public void testDoubleMemChunkLoader() throws IOException { Map doubleTvListMap = buildDoubleTvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(doubleTvListMap); List doubleTvLists = new ArrayList<>(doubleTvListMap.keySet()); + QueryContext queryContext = new QueryContext(false, false); MemPointIterator timeValuePairIterator = - MemPointIteratorFactory.create(TSDataType.DOUBLE, doubleTvLists, maxNumberOfPointsInPage); + MemPointIteratorFactory.create( + TSDataType.DOUBLE, doubleTvLists, maxNumberOfPointsInPage, queryContext); timeValuePairIterator.setStreamingQueryMemChunk(false); timeValuePairIterator.nextBatch(); Mockito.when(chunk.getMemPointIterator()).thenReturn(timeValuePairIterator); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); - MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(false), chunk); + MemChunkLoader memChunkLoader = new MemChunkLoader(queryContext, chunk); try { memChunkLoader.loadChunk(chunkMetadata); fail(); @@ -468,14 +478,16 @@ public void testTextMemChunkLoader() throws IOException { Map textTvListMap = buildTextTvListMap(); Mockito.when(chunk.getTvListQueryMap()).thenReturn(textTvListMap); List textTvLists = new ArrayList<>(textTvListMap.keySet()); + QueryContext queryContext = new QueryContext(false, false); MemPointIterator timeValuePairIterator = - MemPointIteratorFactory.create(TSDataType.TEXT, textTvLists, maxNumberOfPointsInPage); + MemPointIteratorFactory.create( + TSDataType.TEXT, textTvLists, maxNumberOfPointsInPage, queryContext); timeValuePairIterator.setStreamingQueryMemChunk(false); timeValuePairIterator.nextBatch(); Mockito.when(chunk.getMemPointIterator()).thenReturn(timeValuePairIterator); ChunkMetadata chunkMetadata = Mockito.mock(ChunkMetadata.class); - MemChunkLoader memChunkLoader = new MemChunkLoader(new QueryContext(false), chunk); + MemChunkLoader memChunkLoader = new MemChunkLoader(queryContext, chunk); try { memChunkLoader.loadChunk(chunkMetadata); fail(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/TsFilePlanRedoerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/TsFilePlanRedoerTest.java index a86f814041afd..42cbf6eed79c7 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/TsFilePlanRedoerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/file/TsFilePlanRedoerTest.java @@ -162,7 +162,8 @@ public void testRedoInsertRowPlan() throws Exception { new NonAlignedFullPath( DEVICE2_NAME, new MeasurementSchema("s1", TSDataType.FLOAT, TSEncoding.RLE)); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(false), fullPath, Long.MIN_VALUE, null, null); + recoveryMemTable.query( + new QueryContext(false, false), fullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); time = 5; while (iterator.hasNextTimeValuePair()) { @@ -177,7 +178,8 @@ public void testRedoInsertRowPlan() throws Exception { new NonAlignedFullPath( DEVICE2_NAME, new MeasurementSchema("s2", TSDataType.DOUBLE, TSEncoding.RLE)); memChunk = - recoveryMemTable.query(new QueryContext(false), fullPath, Long.MIN_VALUE, null, null); + recoveryMemTable.query( + new QueryContext(false, false), fullPath, Long.MIN_VALUE, null, null); iterator = memChunk.getPointReader(); time = 5; while (iterator.hasNextTimeValuePair()) { @@ -263,7 +265,8 @@ public void testRedoInsertAlignedRowPlan() throws Exception { new MeasurementSchema("s4", TSDataType.FLOAT, TSEncoding.RLE), new MeasurementSchema("s5", TSDataType.TEXT, TSEncoding.PLAIN))); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(false), fullPath, Long.MIN_VALUE, null, null); + recoveryMemTable.query( + new QueryContext(false, false), fullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); int time = 6; while (iterator.hasNextTimeValuePair()) { @@ -343,7 +346,8 @@ public void testRedoInsertTabletPlan() throws Exception { new NonAlignedFullPath( DEVICE1_NAME, new MeasurementSchema("s1", TSDataType.INT32, TSEncoding.RLE)); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(false), fullPath, Long.MIN_VALUE, null, null); + recoveryMemTable.query( + new QueryContext(false, false), fullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); int time = 5; while (iterator.hasNextTimeValuePair()) { @@ -358,7 +362,8 @@ public void testRedoInsertTabletPlan() throws Exception { new NonAlignedFullPath( DEVICE1_NAME, new MeasurementSchema("s2", TSDataType.INT64, TSEncoding.RLE)); memChunk = - recoveryMemTable.query(new QueryContext(false), fullPath, Long.MIN_VALUE, null, null); + recoveryMemTable.query( + new QueryContext(false, false), fullPath, Long.MIN_VALUE, null, null); iterator = memChunk.getPointReader(); time = 5; while (iterator.hasNextTimeValuePair()) { @@ -457,7 +462,8 @@ public void testRedoInsertAlignedTabletPlan() throws Exception { new MeasurementSchema("s4", TSDataType.FLOAT, TSEncoding.RLE), new MeasurementSchema("s5", TSDataType.TEXT, TSEncoding.PLAIN))); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(false), fullPath, Long.MIN_VALUE, null, null); + recoveryMemTable.query( + new QueryContext(false, false), fullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); int time = 6; while (iterator.hasNextTimeValuePair()) { @@ -579,14 +585,16 @@ public void testRedoOverLapPlanIntoUnseqFile() throws Exception { new NonAlignedFullPath( DEVICE1_NAME, new MeasurementSchema("s1", TSDataType.INT32, TSEncoding.RLE)); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(false), fullPath, Long.MIN_VALUE, null, null); + recoveryMemTable.query( + new QueryContext(false, false), fullPath, Long.MIN_VALUE, null, null); assertTrue(memChunk == null || memChunk.isEmpty()); // check d1.s2 fullPath = new NonAlignedFullPath( DEVICE1_NAME, new MeasurementSchema("s2", TSDataType.INT64, TSEncoding.RLE)); memChunk = - recoveryMemTable.query(new QueryContext(false), fullPath, Long.MIN_VALUE, null, null); + recoveryMemTable.query( + new QueryContext(false, false), fullPath, Long.MIN_VALUE, null, null); assertTrue(memChunk == null || memChunk.isEmpty()); } @@ -791,7 +799,8 @@ public void testRedoAlignedInsertAfterDeleteTimeseries() throws Exception { new MeasurementSchema("s4", TSDataType.FLOAT, TSEncoding.RLE), new MeasurementSchema("s5", TSDataType.TEXT, TSEncoding.PLAIN))); ReadOnlyMemChunk memChunk = - recoveryMemTable.query(new QueryContext(false), fullPath, Long.MIN_VALUE, null, null); + recoveryMemTable.query( + new QueryContext(false, false), fullPath, Long.MIN_VALUE, null, null); IPointReader iterator = memChunk.getPointReader(); time = 6; while (iterator.hasNextTimeValuePair()) { diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift index cca7110f28d40..07f5a822e8980 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift @@ -751,6 +751,11 @@ struct TQueryStatistics { 47: i64 loadChunkActualIOSize 48: i64 chunkWithMetadataErrorsCount + + 49: i64 timeSeriesIndexFilteredRows + 50: i64 chunkIndexFilteredRows + 51: i64 pageIndexFilteredRows + 52: i64 rowScanFilteredRows } diff --git a/pom.xml b/pom.xml index 91e2f933a5a81..b8738dc4d84bb 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ 0.14.1 1.9 1.5.6-3 - 2.2.1-260206-SNAPSHOT + 2.2.1-260318-SNAPSHOT