diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/TableWindowOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/TableWindowOperator.java new file mode 100644 index 000000000000..62bf71eb585d --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/TableWindowOperator.java @@ -0,0 +1,411 @@ +/* + * 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.process.window; + +import org.apache.iotdb.db.queryengine.execution.MemoryEstimationHelper; +import org.apache.iotdb.db.queryengine.execution.operator.Operator; +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.execution.operator.process.ProcessOperator; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.WindowFunction; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.FrameInfo; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.RowComparator; +import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; + +import com.google.common.collect.ImmutableList; +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.common.conf.TSFileDescriptor; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; +import static org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanGraphPrinter.MAX_RESERVED_MEMORY; + +public class TableWindowOperator implements ProcessOperator { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(TableWindowOperator.class); + + // Common fields + private final OperatorContext operatorContext; + private final Operator inputOperator; + private final List inputDataTypes; + private final List outputChannels; + private final TsBlockBuilder tsBlockBuilder; + private final MemoryReservationManager memoryReservationManager; + + // Basic information about window operator + private final List windowFunctions; + private final List frameInfoList; + + // Partition + private final List partitionChannels; + private final RowComparator partitionComparator; + private final List cachedTsBlocks; + private int startIndexInFirstBlock; + + // Sort + private final List sortChannels; + + // Transformation + private LinkedList cachedPartitionExecutors; + + // Misc + private long totalMemorySize; + private long maxUsedMemory; + private final long maxRuntime; + + public TableWindowOperator( + OperatorContext operatorContext, + Operator inputOperator, + List inputDataTypes, + List outputDataTypes, + List outputChannels, + List windowFunctions, + List frameInfoList, + List partitionChannels, + List sortChannels) { + // Common part(among all other operators) + this.operatorContext = operatorContext; + this.inputOperator = inputOperator; + this.inputDataTypes = ImmutableList.copyOf(inputDataTypes); + this.outputChannels = ImmutableList.copyOf(outputChannels); + this.tsBlockBuilder = new TsBlockBuilder(outputDataTypes); + + // Basic information part + this.windowFunctions = windowFunctions; + this.frameInfoList = frameInfoList; + + // Partition Part + this.partitionChannels = ImmutableList.copyOf(partitionChannels); + // Acquire partition channels' data types + List partitionDataTypes = new ArrayList<>(); + for (Integer channel : partitionChannels) { + partitionDataTypes.add(inputDataTypes.get(channel)); + } + this.partitionComparator = new RowComparator(partitionDataTypes); + + // Ordering part + this.sortChannels = ImmutableList.copyOf(sortChannels); + + // Transformation part + this.cachedPartitionExecutors = new LinkedList<>(); + + // Misc + this.cachedTsBlocks = new ArrayList<>(); + this.startIndexInFirstBlock = -1; + this.maxRuntime = this.operatorContext.getMaxRunTime().roundTo(TimeUnit.NANOSECONDS); + this.totalMemorySize = 0; + this.maxUsedMemory = 0; + this.memoryReservationManager = + operatorContext + .getDriverContext() + .getFragmentInstanceContext() + .getMemoryReservationContext(); + } + + @Override + public OperatorContext getOperatorContext() { + return operatorContext; + } + + @Override + public TsBlock next() throws Exception { + long startTime = System.nanoTime(); + + // Transform is not finished + if (!cachedPartitionExecutors.isEmpty()) { + TsBlock tsBlock = transform(startTime); + if (tsBlock != null) { + return tsBlock; + } + // Receive more data when result TsBlock builder is not full + // In this case, all partition executors are done + } + + if (inputOperator.hasNextWithTimer()) { + // This TsBlock is pre-sorted with PARTITION BY and ORDER BY channels + TsBlock preSortedBlock = inputOperator.next(); + // StreamSort Operator sometimes returns null + if (preSortedBlock == null || preSortedBlock.isEmpty()) { + return null; + } + + cachedPartitionExecutors = partition(preSortedBlock); + if (cachedPartitionExecutors.isEmpty()) { + // No partition found + // i.e., partition crosses multiple TsBlocks + return null; + } + + // May return null if builder is not full + return transform(startTime); + } else if (!cachedTsBlocks.isEmpty()) { + // Form last partition + TsBlock lastTsBlock = cachedTsBlocks.get(cachedTsBlocks.size() - 1); + int endIndexOfLastTsBlock = lastTsBlock.getPositionCount(); + PartitionExecutor partitionExecutor = + new PartitionExecutor( + cachedTsBlocks, + inputDataTypes, + startIndexInFirstBlock, + endIndexOfLastTsBlock, + outputChannels, + windowFunctions, + frameInfoList, + sortChannels); + cachedPartitionExecutors.addLast(partitionExecutor); + cachedTsBlocks.clear(); + releaseAllCachedTsBlockMemory(); + + TsBlock tsBlock = transform(startTime); + if (tsBlock == null) { + // TsBlockBuilder is not full + // Force build since this is the last partition + tsBlock = + tsBlockBuilder.build( + new RunLengthEncodedColumn( + TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + tsBlockBuilder.reset(); + } + + return tsBlock; + } else if (!tsBlockBuilder.isEmpty()) { + // Return remaining data in result TsBlockBuilder + // This happens when last partition is too large + // And TsBlockBuilder is not full at the end of transform + return getTsBlockFromTsBlockBuilder(); + } + + return null; + } + + private LinkedList partition(TsBlock tsBlock) { + LinkedList partitionExecutors = new LinkedList<>(); + + int partitionStartInCurrentBlock = 0; + int partitionEndInCurrentBlock = partitionStartInCurrentBlock + 1; + + // In this stage, we only consider partition channels + List partitionColumns = extractPartitionColumns(tsBlock); + + // Previous TsBlocks forms a partition + if (!cachedTsBlocks.isEmpty()) { + TsBlock lastTsBlock = cachedTsBlocks.get(cachedTsBlocks.size() - 1); + int endIndexOfLastTsBlock = lastTsBlock.getPositionCount(); + + // Whether the first row of current TsBlock is not equal to + // last row of previous cached TsBlocks + List lastPartitionColumns = extractPartitionColumns(lastTsBlock); + if (!partitionComparator.equal( + partitionColumns, 0, lastPartitionColumns, endIndexOfLastTsBlock - 1)) { + PartitionExecutor partitionExecutor = + new PartitionExecutor( + cachedTsBlocks, + inputDataTypes, + startIndexInFirstBlock, + endIndexOfLastTsBlock, + outputChannels, + windowFunctions, + frameInfoList, + sortChannels); + + partitionExecutors.addLast(partitionExecutor); + cachedTsBlocks.clear(); + releaseAllCachedTsBlockMemory(); + startIndexInFirstBlock = -1; + } + } + + // Try to find all partitions + int count = tsBlock.getPositionCount(); + while (count == 1 || partitionEndInCurrentBlock < count) { + // Try to find one partition + while (partitionEndInCurrentBlock < count + && partitionComparator.equalColumns( + partitionColumns, partitionStartInCurrentBlock, partitionEndInCurrentBlock)) { + partitionEndInCurrentBlock++; + } + + if (partitionEndInCurrentBlock != count) { + // Find partition + PartitionExecutor partitionExecutor; + if (partitionStartInCurrentBlock != 0 || startIndexInFirstBlock == -1) { + // Small partition within this TsBlock + partitionExecutor = + new PartitionExecutor( + Collections.singletonList(tsBlock), + inputDataTypes, + partitionStartInCurrentBlock, + partitionEndInCurrentBlock, + outputChannels, + windowFunctions, + frameInfoList, + sortChannels); + } else { + // Large partition crosses multiple TsBlocks + reserveOneTsBlockMemory(tsBlock); + cachedTsBlocks.add(tsBlock); + partitionExecutor = + new PartitionExecutor( + cachedTsBlocks, + inputDataTypes, + startIndexInFirstBlock, + partitionEndInCurrentBlock, + outputChannels, + windowFunctions, + frameInfoList, + sortChannels); + // Clear TsBlock of last partition + cachedTsBlocks.clear(); + releaseAllCachedTsBlockMemory(); + } + partitionExecutors.addLast(partitionExecutor); + + partitionStartInCurrentBlock = partitionEndInCurrentBlock; + partitionEndInCurrentBlock = partitionStartInCurrentBlock + 1; + } else { + // Last partition of TsBlock + // The beginning of next TsBlock may have rows in this partition + if (startIndexInFirstBlock == -1) { + startIndexInFirstBlock = partitionStartInCurrentBlock; + } + reserveOneTsBlockMemory(tsBlock); + cachedTsBlocks.add(tsBlock); + // For count == 1 + break; + } + } + + return partitionExecutors; + } + + private TsBlock transform(long startTime) { + while (!cachedPartitionExecutors.isEmpty()) { + PartitionExecutor partitionExecutor = cachedPartitionExecutors.getFirst(); + + while (System.nanoTime() - startTime < maxRuntime + && !tsBlockBuilder.isFull() + && partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + if (!partitionExecutor.hasNext()) { + cachedPartitionExecutors.removeFirst(); + } + + if (System.nanoTime() - startTime >= maxRuntime || tsBlockBuilder.isFull()) { + return getTsBlockFromTsBlockBuilder(); + } + } + + // Reach partition end, but builder is not full yet + return null; + } + + private List extractPartitionColumns(TsBlock tsBlock) { + List partitionColumns = new ArrayList<>(partitionChannels.size()); + for (int channel : partitionChannels) { + Column partitionColumn = tsBlock.getColumn(channel); + partitionColumns.add(partitionColumn); + } + return partitionColumns; + } + + private TsBlock getTsBlockFromTsBlockBuilder() { + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + tsBlockBuilder.reset(); + return result; + } + + @Override + public boolean hasNext() throws Exception { + return !cachedPartitionExecutors.isEmpty() + || inputOperator.hasNext() + || !cachedTsBlocks.isEmpty() + || !tsBlockBuilder.isEmpty(); + } + + @Override + public void close() throws Exception { + inputOperator.close(); + if (totalMemorySize != 0) { + memoryReservationManager.releaseMemoryCumulatively(totalMemorySize); + } + } + + @Override + public boolean isFinished() throws Exception { + return !this.hasNextWithTimer(); + } + + private void reserveOneTsBlockMemory(TsBlock tsBlock) { + long reserved = tsBlock.getTotalInstanceSize(); + memoryReservationManager.reserveMemoryCumulatively(reserved); + totalMemorySize += reserved; + maxUsedMemory = Math.max(maxUsedMemory, totalMemorySize); + operatorContext.recordSpecifiedInfo(MAX_RESERVED_MEMORY, Long.toString(maxUsedMemory)); + } + + private void releaseAllCachedTsBlockMemory() { + long released = cachedTsBlocks.stream().mapToInt(TsBlock::getTotalInstanceSize).sum(); + memoryReservationManager.releaseMemoryCumulatively(released); + totalMemorySize -= released; + // No need to update maxUsedMemory + operatorContext.recordSpecifiedInfo(MAX_RESERVED_MEMORY, Long.toString(maxUsedMemory)); + } + + @Override + public long calculateMaxPeekMemory() { + long maxPeekMemoryFromInput = inputOperator.calculateMaxPeekMemoryWithCounter(); + long maxPeekMemoryFromCurrent = + TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); + return Math.max(maxPeekMemoryFromInput, maxPeekMemoryFromCurrent) + + inputOperator.calculateRetainedSizeAfterCallingNext(); + } + + @Override + public long calculateMaxReturnSize() { + return TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); + } + + @Override + public long calculateRetainedSizeAfterCallingNext() { + return inputOperator.calculateRetainedSizeAfterCallingNext(); + } + + @Override + public long ramBytesUsed() { + return INSTANCE_SIZE + + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(inputOperator) + + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext) + + tsBlockBuilder.getRetainedSizeInBytes(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/WindowFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/WindowFunction.java new file mode 100644 index 000000000000..acaf72da8565 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/WindowFunction.java @@ -0,0 +1,45 @@ +/* + * 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.process.window.function; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public interface WindowFunction { + void reset(); + + void transform( + Partition partition, + ColumnBuilder builder, + int index, + int frameStart, + int frameEnd, + int peerGroupStart, + int peerGroupEnd); + + default boolean needPeerGroup() { + return true; + } + + default boolean needFrame() { + return true; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/aggregate/AggregationWindowFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/aggregate/AggregationWindowFunction.java new file mode 100644 index 000000000000..b8f5a95f3fa3 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/aggregate/AggregationWindowFunction.java @@ -0,0 +1,113 @@ +/* + * 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.process.window.function.aggregate; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.WindowFunction; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class AggregationWindowFunction implements WindowFunction { + private final WindowAggregator aggregator; + private int currentStart; + private int currentEnd; + + public AggregationWindowFunction(WindowAggregator aggregator) { + this.aggregator = aggregator; + reset(); + } + + @Override + public void reset() { + aggregator.reset(); + currentStart = -1; + currentEnd = -1; + } + + @Override + public void transform( + Partition partition, + ColumnBuilder builder, + int index, + int frameStart, + int frameEnd, + int peerGroupStart, + int peerGroupEnd) { + if (frameStart < 0) { + // Empty frame + reset(); + } else if (frameStart == currentStart && frameEnd >= currentEnd) { + // Frame expansion + if (frameEnd != currentEnd) { + Partition region = partition.getRegion(currentEnd + 1, frameEnd); + aggregator.addInput(region); + currentEnd = frameEnd; + } + } else { + buildNewFrame(partition, frameStart, frameEnd); + } + + aggregator.evaluate(builder); + } + + private void buildNewFrame(Partition partition, int frameStart, int frameEnd) { + if (aggregator.removable()) { + int prefix = Math.abs(currentStart - frameStart); + int suffix = Math.abs(currentEnd - frameEnd); + int frameLength = frameEnd - frameStart + 1; + + // Compare remove && add cost with re-computation + if (frameLength > prefix + suffix) { + if (currentStart < frameStart) { + Partition region = partition.getRegion(currentStart, frameStart - 1); + aggregator.removeInput(region); + } else if (currentStart > frameStart) { + Partition region = partition.getRegion(frameStart, currentStart - 1); + aggregator.addInput(region); + } // Do nothing when currentStart == frameStart + + if (frameEnd < currentEnd) { + Partition region = partition.getRegion(frameEnd + 1, currentEnd); + aggregator.removeInput(region); + } else if (frameEnd > currentEnd) { + Partition region = partition.getRegion(currentEnd + 1, frameEnd); + aggregator.addInput(region); + } // Do nothing when frameEnd == currentEnd + + currentStart = frameStart; + currentEnd = frameEnd; + return; + } + } + + // Re-compute + aggregator.reset(); + Partition region = partition.getRegion(frameStart, frameEnd); + aggregator.addInput(region); + + currentStart = frameStart; + currentEnd = frameEnd; + } + + @Override + public boolean needPeerGroup() { + return false; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/aggregate/WindowAggregator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/aggregate/WindowAggregator.java new file mode 100644 index 000000000000..5d540db57442 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/aggregate/WindowAggregator.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.execution.operator.process.window.function.aggregate; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.AggregationMask; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.TableAccumulator; + +import com.google.common.primitives.Ints; +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.block.column.ColumnBuilder; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.statistics.Statistics; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; + +import java.util.List; + +import static java.util.Objects.requireNonNull; +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class WindowAggregator { + private final TableAccumulator accumulator; + private final TSDataType outputType; + private final int[] inputChannels; + + public WindowAggregator( + TableAccumulator accumulator, TSDataType outputType, List inputChannels) { + this.accumulator = requireNonNull(accumulator, "accumulator is null"); + this.outputType = requireNonNull(outputType, "intermediateType is null"); + this.inputChannels = Ints.toArray(requireNonNull(inputChannels, "inputChannels is null")); + } + + public TSDataType getType() { + return outputType; + } + + public void addInput(Partition partition) { + List allColumns = partition.getAllColumns(); + for (Column[] columns : allColumns) { + addInput(columns); + } + } + + public void addInput(Column[] columns) { + Column[] arguments = new Column[inputChannels.length]; + for (int i = 0; i < inputChannels.length; i++) { + arguments[i] = columns[inputChannels[i]]; + } + + // Process count(*) + int count = columns[0].getPositionCount(); + if (arguments.length == 0) { + arguments = new Column[] {new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, count)}; + } + + AggregationMask mask = AggregationMask.createSelectAll(count); + accumulator.addInput(arguments, mask); + } + + public void removeInput(Partition partition) { + List allColumns = partition.getAllColumns(); + for (Column[] columns : allColumns) { + removeInput(columns); + } + } + + private void removeInput(Column[] columns) { + Column[] arguments = new Column[inputChannels.length]; + for (int i = 0; i < inputChannels.length; i++) { + arguments[i] = columns[inputChannels[i]]; + } + + // Process count(*) + int count = columns[0].getPositionCount(); + if (arguments.length == 0) { + arguments = new Column[] {new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, count)}; + } + + accumulator.removeInput(arguments); + } + + public void evaluate(ColumnBuilder columnBuilder) { + accumulator.evaluateFinal(columnBuilder); + } + + public void processStatistics(Statistics[] statistics) { + accumulator.addStatistics(statistics); + } + + public boolean hasFinalResult() { + return accumulator.hasFinalResult(); + } + + public void reset() { + accumulator.reset(); + } + + public boolean removable() { + return accumulator.removable(); + } + + public long getEstimatedSize() { + return accumulator.getEstimatedSize(); + } + + public int getChannelCount() { + return this.inputChannels.length; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/CumeDistFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/CumeDistFunction.java new file mode 100644 index 000000000000..e4745bd0ad54 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/CumeDistFunction.java @@ -0,0 +1,52 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class CumeDistFunction extends RankWindowFunction { + private long count; + + public CumeDistFunction() { + reset(); + } + + @Override + public void reset() { + super.reset(); + count = 0; + } + + @Override + public void transform( + Partition partition, + ColumnBuilder builder, + int index, + boolean isNewPeerGroup, + int peerGroupCount) { + if (isNewPeerGroup) { + count += peerGroupCount; + } + + builder.writeDouble(((double) count) / partition.getPositionCount()); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/DenseRankFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/DenseRankFunction.java new file mode 100644 index 000000000000..ac16fcf66f08 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/DenseRankFunction.java @@ -0,0 +1,52 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class DenseRankFunction extends RankWindowFunction { + private long rank; + + public DenseRankFunction() { + reset(); + } + + @Override + public void reset() { + super.reset(); + rank = 0; + } + + @Override + public void transform( + Partition partition, + ColumnBuilder builder, + int index, + boolean isNewPeerGroup, + int peerGroupCount) { + if (isNewPeerGroup) { + rank++; + } + + builder.writeLong(rank); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/NTileFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/NTileFunction.java new file mode 100644 index 000000000000..48b70cc8ba69 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/NTileFunction.java @@ -0,0 +1,62 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class NTileFunction extends RankWindowFunction { + private final int n; + + public NTileFunction(int n) { + this.n = n; + } + + @Override + public void transform( + Partition partition, + ColumnBuilder builder, + int index, + boolean isNewPeerGroup, + int peerGroupCount) { + builder.writeLong(bucket(n, index, partition.getPositionCount()) + 1); + } + + private long bucket(long buckets, int index, int count) { + if (count < buckets) { + return index; + } + + long remainderRows = count % buckets; + long rowsPerBucket = count / buckets; + + if (index < ((rowsPerBucket + 1) * remainderRows)) { + return index / (rowsPerBucket + 1); + } + + return (index - remainderRows) / rowsPerBucket; + } + + @Override + public boolean needPeerGroup() { + return false; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/PercentRankFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/PercentRankFunction.java new file mode 100644 index 000000000000..35741eb4078b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/PercentRankFunction.java @@ -0,0 +1,63 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class PercentRankFunction extends RankWindowFunction { + private long rank; + private long count; + + public PercentRankFunction() { + reset(); + } + + @Override + public void reset() { + super.reset(); + rank = 0; + count = 1; + } + + @Override + public void transform( + Partition partition, + ColumnBuilder builder, + int index, + boolean isNewPeerGroup, + int peerGroupCount) { + int total = partition.getPositionCount(); + if (total == 1) { + builder.writeDouble(0); + return; + } + + if (isNewPeerGroup) { + rank += count; + count = 1; + } else { + count++; + } + + builder.writeDouble(((double) (rank - 1)) / (total - 1)); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RankFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RankFunction.java new file mode 100644 index 000000000000..c6fbf710e725 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RankFunction.java @@ -0,0 +1,57 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class RankFunction extends RankWindowFunction { + private long rank; + private long count; + + public RankFunction() { + reset(); + } + + @Override + public void reset() { + super.reset(); + rank = 0; + count = 1; + } + + @Override + public void transform( + Partition partition, + ColumnBuilder builder, + int index, + boolean isNewPeerGroup, + int peerGroupCount) { + if (isNewPeerGroup) { + rank += count; + count = 1; + } else { + count++; + } + + builder.writeLong(rank); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RankWindowFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RankWindowFunction.java new file mode 100644 index 000000000000..b2c672afbea6 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RankWindowFunction.java @@ -0,0 +1,66 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.WindowFunction; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public abstract class RankWindowFunction implements WindowFunction { + private int currentPeerGroupStart; + + @Override + public void reset() { + currentPeerGroupStart = -1; + } + + @Override + public void transform( + Partition partition, + ColumnBuilder builder, + int index, + int frameStart, + int frameEnd, + int peerGroupStart, + int peerGroupEnd) { + boolean isNewPeerGroup = false; + if (peerGroupStart != currentPeerGroupStart) { + currentPeerGroupStart = peerGroupStart; + isNewPeerGroup = true; + } + + int peerGroupCount = (peerGroupEnd - peerGroupStart) + 1; + + transform(partition, builder, index, isNewPeerGroup, peerGroupCount); + } + + public abstract void transform( + Partition partition, + ColumnBuilder builder, + int index, + boolean isNewPeerGroup, + int peerGroupCount); + + @Override + public boolean needFrame() { + return false; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RowNumberFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RowNumberFunction.java new file mode 100644 index 000000000000..67a790557bcb --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RowNumberFunction.java @@ -0,0 +1,41 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class RowNumberFunction extends RankWindowFunction { + @Override + public void transform( + Partition partition, + ColumnBuilder builder, + int index, + boolean isNewPeerGroup, + int peerGroupCount) { + builder.writeLong((long) index + 1); + } + + @Override + public boolean needPeerGroup() { + return false; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/FirstValueFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/FirstValueFunction.java new file mode 100644 index 000000000000..465ec2920fc2 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/FirstValueFunction.java @@ -0,0 +1,64 @@ +/* + * 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.process.window.function.value; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class FirstValueFunction extends ValueWindowFunction { + private final int channel; + private final boolean ignoreNull; + + public FirstValueFunction(int channel, boolean ignoreNull) { + this.channel = channel; + this.ignoreNull = ignoreNull; + } + + @Override + public void transform( + Partition partition, ColumnBuilder builder, int index, int frameStart, int frameEnd) { + // Empty frame + if (frameStart < 0) { + builder.appendNull(); + return; + } + + if (ignoreNull) { + // Handle nulls + int pos = frameStart; + while (pos <= frameEnd && partition.isNull(channel, pos)) { + pos++; + } + + if (pos > frameEnd) { + builder.appendNull(); + } else { + partition.writeTo(builder, channel, pos); + } + } else { + if (partition.isNull(channel, frameStart)) { + builder.appendNull(); + } else { + partition.writeTo(builder, channel, frameStart); + } + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LagFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LagFunction.java new file mode 100644 index 000000000000..cfa48742f924 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LagFunction.java @@ -0,0 +1,77 @@ +/* + * 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.process.window.function.value; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class LagFunction extends ValueWindowFunction { + private final int channel; + private final Integer offset; + private final Object defaultVal; + private final boolean ignoreNull; + + public LagFunction(int channel, Integer offset, Object defaultVal, boolean ignoreNull) { + this.channel = channel; + this.offset = offset == null ? 1 : offset; + this.defaultVal = defaultVal; + this.ignoreNull = ignoreNull; + } + + @Override + public void transform( + Partition partition, ColumnBuilder builder, int index, int frameStart, int frameEnd) { + int pos; + if (ignoreNull) { + int nonNullCount = 0; + pos = index - 1; + while (pos >= 0) { + if (!partition.isNull(channel, pos)) { + nonNullCount++; + if (nonNullCount == offset) { + break; + } + } + + pos--; + } + } else { + pos = index - offset; + } + + if (pos >= 0) { + if (!partition.isNull(channel, pos)) { + partition.writeTo(builder, channel, pos); + } else { + builder.appendNull(); + } + } else if (defaultVal != null) { + builder.writeObject(defaultVal); + } else { + builder.appendNull(); + } + } + + @Override + public boolean needFrame() { + return false; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LastValueFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LastValueFunction.java new file mode 100644 index 000000000000..1c929d035be3 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LastValueFunction.java @@ -0,0 +1,64 @@ +/* + * 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.process.window.function.value; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class LastValueFunction extends ValueWindowFunction { + private final int channel; + private final boolean ignoreNull; + + public LastValueFunction(int channel, boolean ignoreNull) { + this.channel = channel; + this.ignoreNull = ignoreNull; + } + + @Override + public void transform( + Partition partition, ColumnBuilder builder, int index, int frameStart, int frameEnd) { + // Empty frame + if (frameStart < 0) { + builder.appendNull(); + return; + } + + if (ignoreNull) { + // Handle nulls + int pos = frameEnd; + while (pos >= frameStart && partition.isNull(channel, pos)) { + pos--; + } + + if (pos < frameStart) { + builder.appendNull(); + } else { + partition.writeTo(builder, channel, pos); + } + } else { + if (partition.isNull(channel, frameEnd)) { + builder.appendNull(); + } else { + partition.writeTo(builder, channel, frameEnd); + } + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LeadFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LeadFunction.java new file mode 100644 index 000000000000..514357df57a9 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LeadFunction.java @@ -0,0 +1,79 @@ +/* + * 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.process.window.function.value; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class LeadFunction extends ValueWindowFunction { + private final int channel; + private final Integer offset; + private final Integer defaultVal; + private final boolean ignoreNull; + + public LeadFunction(int channel, Integer offset, Integer defaultVal, boolean ignoreNull) { + this.channel = channel; + this.offset = offset == null ? 1 : offset; + this.defaultVal = defaultVal; + this.ignoreNull = ignoreNull; + } + + @Override + public void transform( + Partition partition, ColumnBuilder builder, int index, int frameStart, int frameEnd) { + int length = partition.getPositionCount(); + + int pos; + if (ignoreNull) { + int nonNullCount = 0; + pos = index + 1; + while (pos < length) { + if (!partition.isNull(channel, pos)) { + nonNullCount++; + if (nonNullCount == offset) { + break; + } + } + + pos++; + } + } else { + pos = index + offset; + } + + if (pos < length) { + if (!partition.isNull(channel, pos)) { + partition.writeTo(builder, channel, pos); + } else { + builder.appendNull(); + } + } else if (defaultVal != null) { + builder.writeObject(defaultVal); + } else { + builder.appendNull(); + } + } + + @Override + public boolean needFrame() { + return false; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/NthValueFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/NthValueFunction.java new file mode 100644 index 000000000000..8978b5f13b61 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/NthValueFunction.java @@ -0,0 +1,81 @@ +/* + * 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.process.window.function.value; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public class NthValueFunction extends ValueWindowFunction { + private final int n; + private final int channel; + private final boolean ignoreNull; + + public NthValueFunction(int n, int channel, boolean ignoreNull) { + this.n = n; + this.channel = channel; + this.ignoreNull = ignoreNull; + } + + @Override + public void transform( + Partition partition, ColumnBuilder builder, int index, int frameStart, int frameEnd) { + // Empty frame + if (frameStart < 0) { + builder.appendNull(); + return; + } + + int pos; + if (ignoreNull) { + // Handle nulls + pos = frameStart; + int nonNullCount = 0; + while (pos <= frameEnd) { + if (!partition.isNull(channel, pos)) { + nonNullCount++; + if (nonNullCount == n) { + break; + } + } + pos++; + } + + if (pos <= frameEnd) { + partition.writeTo(builder, channel, pos); + } else { + builder.appendNull(); + } + return; + } + + // n starts with 1 + pos = frameStart + n - 1; + if (pos <= frameEnd) { + if (!partition.isNull(channel, pos)) { + partition.writeTo(builder, channel, pos); + } else { + builder.appendNull(); + } + } else { + builder.appendNull(); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/ValueWindowFunction.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/ValueWindowFunction.java new file mode 100644 index 000000000000..45e5b9def92b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/ValueWindowFunction.java @@ -0,0 +1,52 @@ +/* + * 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.process.window.function.value; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.WindowFunction; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; + +import org.apache.tsfile.block.column.ColumnBuilder; + +public abstract class ValueWindowFunction implements WindowFunction { + @Override + public void reset() { + // do nothing, value functions are stateless + } + + @Override + public void transform( + Partition partition, + ColumnBuilder builder, + int index, + int frameStart, + int frameEnd, + int peerGroupStart, + int peerGroupEnd) { + transform(partition, builder, index, frameStart, frameEnd); + } + + public abstract void transform( + Partition partition, ColumnBuilder builder, int index, int frameStart, int frameEnd); + + @Override + public boolean needPeerGroup() { + return false; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/Partition.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/Partition.java new file mode 100644 index 000000000000..b8acd7fae11f --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/Partition.java @@ -0,0 +1,226 @@ +/* + * 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.process.window.partition; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.ColumnList; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.block.column.ColumnBuilder; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.utils.Binary; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class Partition { + private final List tsBlocks; + private int cachedPositionCount = -1; + + public Partition(List tsBlocks, int startIndexInFirstBlock, int endIndexInLastBlock) { + if (tsBlocks.size() == 1) { + int length = endIndexInLastBlock - startIndexInFirstBlock; + this.tsBlocks = + Collections.singletonList(tsBlocks.get(0).getRegion(startIndexInFirstBlock, length)); + return; + } + + this.tsBlocks = new ArrayList<>(tsBlocks.size()); + // First TsBlock + TsBlock firstBlock = tsBlocks.get(0).subTsBlock(startIndexInFirstBlock); + this.tsBlocks.add(firstBlock); + // Middle TsBlock + for (int i = 1; i < tsBlocks.size() - 1; i++) { + this.tsBlocks.add(tsBlocks.get(i)); + } + // Last TsBlock + TsBlock lastBlock = tsBlocks.get(tsBlocks.size() - 1).getRegion(0, endIndexInLastBlock); + this.tsBlocks.add(lastBlock); + } + + public int getPositionCount() { + if (cachedPositionCount == -1) { + // Lazy initialized + cachedPositionCount = 0; + for (TsBlock block : tsBlocks) { + cachedPositionCount += block.getPositionCount(); + } + } + + return cachedPositionCount; + } + + public int getValueColumnCount() { + return tsBlocks.get(0).getValueColumnCount(); + } + + public TsBlock getTsBlock(int tsBlockIndex) { + return tsBlocks.get(tsBlockIndex); + } + + public List getAllColumns() { + List allColumns = new ArrayList<>(); + for (TsBlock block : tsBlocks) { + allColumns.add(block.getAllColumns()); + } + + return allColumns; + } + + public boolean getBoolean(int channel, int rowIndex) { + PartitionIndex partitionIndex = getPartitionIndex(rowIndex); + int tsBlockIndex = partitionIndex.getTsBlockIndex(); + int offsetInTsBlock = partitionIndex.getOffsetInTsBlock(); + + TsBlock tsBlock = tsBlocks.get(tsBlockIndex); + return tsBlock.getColumn(channel).getBoolean(offsetInTsBlock); + } + + public int getInt(int channel, int rowIndex) { + PartitionIndex partitionIndex = getPartitionIndex(rowIndex); + int tsBlockIndex = partitionIndex.getTsBlockIndex(); + int offsetInTsBlock = partitionIndex.getOffsetInTsBlock(); + + TsBlock tsBlock = tsBlocks.get(tsBlockIndex); + return tsBlock.getColumn(channel).getInt(offsetInTsBlock); + } + + public long getLong(int channel, int rowIndex) { + PartitionIndex partitionIndex = getPartitionIndex(rowIndex); + int tsBlockIndex = partitionIndex.getTsBlockIndex(); + int offsetInTsBlock = partitionIndex.getOffsetInTsBlock(); + + TsBlock tsBlock = tsBlocks.get(tsBlockIndex); + return tsBlock.getColumn(channel).getLong(offsetInTsBlock); + } + + public float getFloat(int channel, int rowIndex) { + PartitionIndex partitionIndex = getPartitionIndex(rowIndex); + int tsBlockIndex = partitionIndex.getTsBlockIndex(); + int offsetInTsBlock = partitionIndex.getOffsetInTsBlock(); + + TsBlock tsBlock = tsBlocks.get(tsBlockIndex); + return tsBlock.getColumn(channel).getFloat(offsetInTsBlock); + } + + public double getDouble(int channel, int rowIndex) { + PartitionIndex partitionIndex = getPartitionIndex(rowIndex); + int tsBlockIndex = partitionIndex.getTsBlockIndex(); + int offsetInTsBlock = partitionIndex.getOffsetInTsBlock(); + + TsBlock tsBlock = tsBlocks.get(tsBlockIndex); + return tsBlock.getColumn(channel).getDouble(offsetInTsBlock); + } + + public Binary getBinary(int channel, int rowIndex) { + PartitionIndex partitionIndex = getPartitionIndex(rowIndex); + int tsBlockIndex = partitionIndex.getTsBlockIndex(); + int offsetInTsBlock = partitionIndex.getOffsetInTsBlock(); + + TsBlock tsBlock = tsBlocks.get(tsBlockIndex); + return tsBlock.getColumn(channel).getBinary(offsetInTsBlock); + } + + public boolean isNull(int channel, int rowIndex) { + PartitionIndex partitionIndex = getPartitionIndex(rowIndex); + int tsBlockIndex = partitionIndex.getTsBlockIndex(); + int offsetInTsBlock = partitionIndex.getOffsetInTsBlock(); + + TsBlock tsBlock = tsBlocks.get(tsBlockIndex); + return tsBlock.getColumn(channel).isNull(offsetInTsBlock); + } + + public void writeTo(ColumnBuilder builder, int channel, int rowIndex) { + PartitionIndex partitionIndex = getPartitionIndex(rowIndex); + int tsBlockIndex = partitionIndex.getTsBlockIndex(); + int offsetInTsBlock = partitionIndex.getOffsetInTsBlock(); + + Column column = tsBlocks.get(tsBlockIndex).getColumn(channel); + builder.write(column, offsetInTsBlock); + } + + public static class PartitionIndex { + private final int tsBlockIndex; + private final int offsetInTsBlock; + + PartitionIndex(int tsBlockIndex, int offsetInTsBlock) { + this.tsBlockIndex = tsBlockIndex; + this.offsetInTsBlock = offsetInTsBlock; + } + + public int getTsBlockIndex() { + return tsBlockIndex; + } + + public int getOffsetInTsBlock() { + return offsetInTsBlock; + } + } + + // start and end are indexes within partition + // Both of them are inclusive, i.e. [start, end] + public Partition getRegion(int start, int end) { + PartitionIndex startPartitionIndex = getPartitionIndex(start); + PartitionIndex endPartitionIndex = getPartitionIndex(end); + + List tsBlockList = new ArrayList<>(); + int startTsBlockIndex = startPartitionIndex.getTsBlockIndex(); + int endTsBlockIndex = endPartitionIndex.getTsBlockIndex(); + for (int i = startTsBlockIndex; i <= endTsBlockIndex; i++) { + tsBlockList.add(tsBlocks.get(i)); + } + + int startIndexInFirstBlock = startPartitionIndex.getOffsetInTsBlock(); + int endIndexInLastBlock = endPartitionIndex.getOffsetInTsBlock(); + return new Partition(tsBlockList, startIndexInFirstBlock, endIndexInLastBlock + 1); + } + + // rowIndex is index within partition + public PartitionIndex getPartitionIndex(int rowIndex) { + int tsBlockIndex = 0; + while (tsBlockIndex < tsBlocks.size() + && rowIndex >= tsBlocks.get(tsBlockIndex).getPositionCount()) { + rowIndex -= tsBlocks.get(tsBlockIndex).getPositionCount(); + // Enter next TsBlock + tsBlockIndex++; + } + + if (tsBlockIndex != tsBlocks.size()) { + return new PartitionIndex(tsBlockIndex, rowIndex); + } else { + // Unlikely + throw new IndexOutOfBoundsException("Index out of Partition's bounds!"); + } + } + + public List getSortedColumnList(List sortedChannels) { + List columnLists = new ArrayList<>(); + + for (Integer sortedChannel : sortedChannels) { + List columns = new ArrayList<>(); + for (TsBlock tsBlock : tsBlocks) { + columns.add(tsBlock.getColumn(sortedChannel)); + } + columnLists.add(new ColumnList(columns)); + } + + return columnLists; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/PartitionExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/PartitionExecutor.java new file mode 100644 index 000000000000..b7a4239a314b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/PartitionExecutor.java @@ -0,0 +1,197 @@ +/* + * 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.process.window.partition; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.WindowFunction; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.Frame; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.FrameInfo; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.GroupsFrame; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.RangeFrame; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.RowsFrame; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.ColumnList; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.Range; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.RowComparator; + +import com.google.common.collect.ImmutableList; +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.block.column.ColumnBuilder; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; + +import java.util.ArrayList; +import java.util.List; + +public final class PartitionExecutor { + private final int partitionStart; + private final int partitionEnd; + private final Partition partition; + + private final List windowFunctions; + + private final List sortedColumns; + private final RowComparator peerGroupComparator; + private int peerGroupStart; + private int peerGroupEnd; + + private final List outputChannels; + + private int currentGroupIndex = -1; + private int currentPosition; + + private final List frames; + + private final boolean needPeerGroup; + + public PartitionExecutor( + List tsBlocks, + List dataTypes, + int startIndexInFirstBlock, + int endIndexInLastBlock, + List outputChannels, + List windowFunctions, + List frameInfoList, + List sortChannels) { + // Partition + this.partition = new Partition(tsBlocks, startIndexInFirstBlock, endIndexInLastBlock); + this.partitionStart = startIndexInFirstBlock; + this.partitionEnd = startIndexInFirstBlock + this.partition.getPositionCount(); + // Window functions and frames + this.windowFunctions = ImmutableList.copyOf(windowFunctions); + this.frames = new ArrayList<>(); + + this.outputChannels = ImmutableList.copyOf(outputChannels); + + // Prepare for peer group comparing + List sortDataTypes = new ArrayList<>(); + for (int channel : sortChannels) { + TSDataType dataType = dataTypes.get(channel); + sortDataTypes.add(dataType); + } + peerGroupComparator = new RowComparator(sortDataTypes); + sortedColumns = partition.getSortedColumnList(sortChannels); + + // Reset functions for new partition + for (WindowFunction windowFunction : windowFunctions) { + windowFunction.reset(); + } + + currentPosition = partitionStart; + needPeerGroup = + windowFunctions.stream().anyMatch(WindowFunction::needPeerGroup) + || frameInfoList.stream() + .anyMatch(frameInfo -> frameInfo.getFrameType() != FrameInfo.FrameType.ROWS); + if (needPeerGroup) { + updatePeerGroup(); + } + + for (int i = 0; i < frameInfoList.size(); i++) { + Frame frame = null; + if (windowFunctions.get(i).needFrame()) { + FrameInfo frameInfo = frameInfoList.get(i); + switch (frameInfo.getFrameType()) { + case RANGE: + frame = new RangeFrame(partition, frameInfo, sortedColumns, peerGroupComparator); + break; + case ROWS: + frame = new RowsFrame(partition, frameInfo, partitionStart, partitionEnd); + break; + case GROUPS: + frame = + new GroupsFrame( + partition, + frameInfo, + sortedColumns, + peerGroupComparator, + peerGroupEnd - partitionStart - 1); + break; + default: + // Unreachable + throw new UnsupportedOperationException("Unreachable!"); + } + } + frames.add(frame); + } + } + + public boolean hasNext() { + return currentPosition < partitionEnd; + } + + public void processNextRow(TsBlockBuilder builder) { + // Copy origin data + int index = currentPosition - partitionStart; + Partition.PartitionIndex partitionIndex = partition.getPartitionIndex(index); + int tsBlockIndex = partitionIndex.getTsBlockIndex(); + int offsetInTsBlock = partitionIndex.getOffsetInTsBlock(); + TsBlock tsBlock = partition.getTsBlock(tsBlockIndex); + + int channel = 0; + for (int i = 0; i < outputChannels.size(); i++) { + Column column = tsBlock.getColumn(outputChannels.get(i)); + ColumnBuilder columnBuilder = builder.getColumnBuilder(i); + columnBuilder.write(column, offsetInTsBlock); + channel++; + } + + if (needPeerGroup && currentPosition == peerGroupEnd) { + updatePeerGroup(); + } + + for (int i = 0; i < windowFunctions.size(); i++) { + Frame frame = frames.get(i); + WindowFunction windowFunction = windowFunctions.get(i); + + Range frameRange = + windowFunction.needFrame() + ? frame.getRange( + index, + currentGroupIndex, + peerGroupStart - partitionStart, + peerGroupEnd - partitionStart) + : new Range(-1, -1); + windowFunction.transform( + partition, + builder.getColumnBuilder(channel), + currentPosition - partitionStart, + frameRange.getStart(), + frameRange.getEnd(), + peerGroupStart - partitionStart, + peerGroupEnd - partitionStart - 1); + + channel++; + } + + currentPosition++; + builder.declarePosition(); + } + + private void updatePeerGroup() { + currentGroupIndex++; + peerGroupStart = currentPosition; + // Find end of peer group + peerGroupEnd = peerGroupStart + 1; + while (peerGroupEnd < partitionEnd + && peerGroupComparator.equalColumnLists( + sortedColumns, peerGroupStart - partitionStart, peerGroupEnd - partitionStart)) { + peerGroupEnd++; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/Frame.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/Frame.java new file mode 100644 index 000000000000..b697420443c5 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/Frame.java @@ -0,0 +1,26 @@ +/* + * 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.process.window.partition.frame; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.Range; + +public interface Frame { + Range getRange(int currentPosition, int currentGroup, int peerGroupStart, int peerGroupEnd); +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/FrameInfo.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/FrameInfo.java new file mode 100644 index 000000000000..eac6126e4981 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/FrameInfo.java @@ -0,0 +1,145 @@ +/* + * 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.process.window.partition.frame; + +import org.apache.iotdb.db.queryengine.plan.relational.planner.SortOrder; + +public class FrameInfo { + public enum FrameType { + RANGE, + ROWS, + GROUPS + } + + public enum FrameBoundType { + UNBOUNDED_PRECEDING, + PRECEDING, + CURRENT_ROW, + FOLLOWING, + UNBOUNDED_FOLLOWING; + } + + private final FrameType frameType; + private final FrameBoundType startType; + private final int startOffsetChannel; // For PRECEDING and FOLLOWING use + private final FrameBoundType endType; + private final int endOffsetChannel; // Same as startOffset + // For RANGE type frame + private final int sortChannel; + private final SortOrder sortOrder; + + public FrameInfo(FrameType frameType, FrameBoundType startType, FrameBoundType endType) { + this(frameType, startType, -1, endType, -1); + } + + public FrameInfo( + FrameType frameType, + FrameBoundType startType, + int startOffsetChannel, + FrameBoundType endType) { + this(frameType, startType, startOffsetChannel, endType, -1); + } + + public FrameInfo( + FrameType frameType, + FrameBoundType startType, + int startOffsetChannel, + FrameBoundType endType, + int sortChannel, + SortOrder sortOrder) { + this(frameType, startType, startOffsetChannel, endType, -1, sortChannel, sortOrder); + } + + public FrameInfo( + FrameType frameType, FrameBoundType startType, FrameBoundType endType, int endOffsetChannel) { + this(frameType, startType, -1, endType, endOffsetChannel); + } + + public FrameInfo( + FrameType frameType, + FrameBoundType startType, + FrameBoundType endType, + int endOffsetChannel, + int sortChannel, + SortOrder sortOrder) { + this(frameType, startType, -1, endType, endOffsetChannel, sortChannel, sortOrder); + } + + public FrameInfo( + FrameType frameType, + FrameBoundType startType, + int startOffsetChannel, + FrameBoundType endType, + int endOffsetChannel) { + this( + frameType, + startType, + startOffsetChannel, + endType, + endOffsetChannel, + -1, + SortOrder.ASC_NULLS_FIRST); + } + + public FrameInfo( + FrameType frameType, + FrameBoundType startType, + int startOffsetChannel, + FrameBoundType endType, + int endOffsetChannel, + int sortChannel, + SortOrder sortOrder) { + this.frameType = frameType; + this.startType = startType; + this.startOffsetChannel = startOffsetChannel; + this.endType = endType; + this.endOffsetChannel = endOffsetChannel; + this.sortChannel = sortChannel; + this.sortOrder = sortOrder; + } + + public FrameType getFrameType() { + return frameType; + } + + public FrameBoundType getStartType() { + return startType; + } + + public FrameBoundType getEndType() { + return endType; + } + + public int getStartOffsetChannel() { + return startOffsetChannel; + } + + public int getEndOffsetChannel() { + return endOffsetChannel; + } + + public int getSortChannel() { + return sortChannel; + } + + public SortOrder getSortOrder() { + return sortOrder; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/GroupsFrame.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/GroupsFrame.java new file mode 100644 index 000000000000..3e9b111301da --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/GroupsFrame.java @@ -0,0 +1,238 @@ +/* + * 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.process.window.partition.frame; + +import org.apache.iotdb.db.exception.sql.SemanticException; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.ColumnList; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.Range; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.RowComparator; + +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; + +public class GroupsFrame implements Frame { + private final Partition partition; + private final FrameInfo frameInfo; + private final int partitionSize; + + private final List columns; + private final RowComparator peerGroupComparator; + + private Range recentRange; + private int recentStartPeerGroup; + private int recentEndPeerGroup; + private boolean frameStartFollowingReachEnd = false; + + public GroupsFrame( + Partition partition, + FrameInfo frameInfo, + List sortedColumns, + RowComparator peerGroupComparator, + int initialEnd) { + this.partition = partition; + this.frameInfo = frameInfo; + this.partitionSize = partition.getPositionCount(); + this.columns = sortedColumns; + this.peerGroupComparator = peerGroupComparator; + + this.recentRange = new Range(0, initialEnd); + this.recentStartPeerGroup = 0; + this.recentEndPeerGroup = 0; + } + + @Override + public Range getRange( + int currentPosition, int currentGroup, int peerGroupStart, int peerGroupEnd) { + int frameStart; + switch (frameInfo.getStartType()) { + case UNBOUNDED_PRECEDING: + frameStart = 0; + break; + case PRECEDING: + frameStart = getStartPrecedingOffset(currentPosition, currentGroup); + break; + case CURRENT_ROW: + frameStart = peerGroupStart; + break; + case FOLLOWING: + frameStart = getStartFollowingOffset(currentPosition, currentGroup); + break; + default: + // UNBOUND_FOLLOWING is not allowed in frame start + throw new SemanticException("UNBOUND FOLLOWING is not allowed in frame start!"); + } + + int frameEnd; + switch (frameInfo.getEndType()) { + case PRECEDING: + frameEnd = getEndPrecedingOffset(currentPosition, currentGroup); + break; + case CURRENT_ROW: + frameEnd = peerGroupEnd - 1; + break; + case FOLLOWING: + frameEnd = getEndFollowingOffset(currentPosition, currentGroup); + break; + case UNBOUNDED_FOLLOWING: + frameEnd = partitionSize - 1; + break; + default: + // UNBOUND_PRECEDING is not allowed in frame end + throw new SemanticException("UNBOUND PRECEDING is not allowed in frame end!"); + } + + // Empty frame + if (frameEnd < frameStart || frameEnd < 0 || frameStart >= partitionSize) { + return new Range(-1, -1); + } + + frameStart = Math.max(frameStart, 0); + frameEnd = Math.min(frameEnd, partitionSize - 1); + recentRange = new Range(frameStart, frameEnd); + return recentRange; + } + + private int getStartPrecedingOffset(int currentPosition, int currentGroup) { + int start = recentRange.getStart(); + int offset = (int) getOffset(frameInfo.getStartOffsetChannel(), currentPosition); + + // We may encounter empty frame + if (currentGroup - offset < 0) { + return -1; + } + + if (currentGroup - offset > recentStartPeerGroup) { + int count = currentGroup - offset - recentStartPeerGroup; + for (int i = 0; i < count; i++) { + // Scan over current peer group + start = scanPeerGroup(start); + // Enter next peer group(won't reach partition end) + start++; + } + recentStartPeerGroup = currentGroup - offset; + } + + return start; + } + + private int getEndPrecedingOffset(int currentPosition, int currentGroup) { + int end = recentRange.getEnd(); + int offset = (int) getOffset(frameInfo.getEndOffsetChannel(), currentPosition); + + // We may encounter empty frame + if (currentGroup - offset < 0) { + return -1; + } + + if (currentGroup - offset > recentEndPeerGroup) { + int count = currentGroup - offset - recentEndPeerGroup; + for (int i = 0; i < count; i++) { + // Enter next peer group + end++; + // Scan over current peer group(won't reach partition end) + end = scanPeerGroup(end); + } + recentEndPeerGroup = currentGroup - offset; + } + + return end; + } + + private int getStartFollowingOffset(int currentPosition, int currentGroup) { + // Shortcut if we have reached last peer group already + if (frameStartFollowingReachEnd) { + return partitionSize; + } + + int start = recentRange.getStart(); + + int offset = (int) getOffset(frameInfo.getStartOffsetChannel(), currentPosition); + if (currentGroup + offset > recentStartPeerGroup) { + int count = currentGroup + offset - recentStartPeerGroup; + for (int i = 0; i < count; i++) { + // Scan over current peer group + start = scanPeerGroup(start); + // Enter next peer group + if (start == partitionSize - 1) { + // Reach partition end + // We may encounter empty frame here + recentStartPeerGroup = currentGroup + i; + frameStartFollowingReachEnd = true; + return partitionSize; + } else { + start++; + } + } + recentStartPeerGroup = currentGroup + offset; + } + + return start; + } + + private int getEndFollowingOffset(int currentPosition, int currentGroup) { + int end = recentRange.getEnd(); + // Shortcut if we have reached partition end already + if (end == partitionSize - 1) { + return end; + } + + int offset = (int) getOffset(frameInfo.getEndOffsetChannel(), currentPosition); + if (currentGroup + offset > recentEndPeerGroup) { + int count = currentGroup + offset - recentEndPeerGroup; + for (int i = 0; i < count; i++) { + // Enter next peer group + if (end == partitionSize - 1) { + if (i != count - 1) { + // Too early, we may encounter empty frame + return partitionSize; + } + + // Reach partition end + recentEndPeerGroup = currentGroup + i; + return end; + } + end++; + // Scan over current peer group + end = scanPeerGroup(end); + } + recentEndPeerGroup = currentGroup + offset; + } + + return end; + } + + private int scanPeerGroup(int currentPosition) { + while (currentPosition < partitionSize - 1 + && peerGroupComparator.equalColumnLists(columns, currentPosition, currentPosition + 1)) { + currentPosition++; + } + return currentPosition; + } + + public long getOffset(int channel, int index) { + checkArgument(!partition.isNull(channel, index)); + long offset = partition.getLong(channel, index); + + checkArgument(offset >= 0); + return offset; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RangeFrame.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RangeFrame.java new file mode 100644 index 000000000000..59e9de731483 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RangeFrame.java @@ -0,0 +1,617 @@ +/* + * 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.process.window.partition.frame; + +import org.apache.iotdb.db.exception.sql.SemanticException; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.ColumnList; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.Range; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.RowComparator; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.write.UnSupportedDataTypeException; + +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.FrameInfo.FrameBoundType.CURRENT_ROW; +import static org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING; +import static org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING; + +public class RangeFrame implements Frame { + private final Partition partition; + private final FrameInfo frameInfo; + private final ColumnList column; + private final TSDataType dataType; + + private final int partitionSize; + private final RowComparator peerGroupComparator; + private Range recentRange; + + public RangeFrame( + Partition partition, + FrameInfo frameInfo, + List sortedColumns, + RowComparator comparator) { + this.partition = partition; + this.frameInfo = frameInfo; + // Only one sort key is allowed in range frame + checkArgument(sortedColumns.size() == 1); + this.column = sortedColumns.get(0); + this.dataType = column.getDataType(); + this.partitionSize = partition.getPositionCount(); + this.peerGroupComparator = comparator; + this.recentRange = new Range(0, 0); + } + + @Override + public Range getRange( + int currentPosition, int currentGroup, int peerGroupStart, int peerGroupEnd) { + // Full partition + if (frameInfo.getStartType() == UNBOUNDED_PRECEDING + && frameInfo.getEndType() == UNBOUNDED_FOLLOWING) { + return new Range(0, partitionSize - 1); + } + + // Peer group + if (frameInfo.getStartType() == CURRENT_ROW && frameInfo.getEndType() == CURRENT_ROW + || frameInfo.getStartType() == CURRENT_ROW && frameInfo.getEndType() == UNBOUNDED_FOLLOWING + || frameInfo.getStartType() == UNBOUNDED_PRECEDING + && frameInfo.getEndType() == CURRENT_ROW) { + if (currentPosition == 0 + || !peerGroupComparator.equal(column, currentPosition - 1, currentPosition)) { + // New peer group + int frameStart = frameInfo.getStartType() == CURRENT_ROW ? peerGroupStart : 0; + int frameEnd = frameInfo.getEndType() == CURRENT_ROW ? peerGroupEnd - 1 : partitionSize - 1; + + recentRange = new Range(frameStart, frameEnd); + } + // Old peer group is considered as well + return recentRange; + } + + // Current row is NULL + // According to Spec, behavior of "X PRECEDING", "X FOLLOWING" frame boundaries is similar to + // "CURRENT ROW" for null values. + if (column.isNull(currentPosition)) { + recentRange = + new Range( + frameInfo.getStartType() == UNBOUNDED_PRECEDING ? 0 : peerGroupStart, + frameInfo.getEndType() == UNBOUNDED_FOLLOWING ? partitionSize - 1 : peerGroupEnd - 1); + return recentRange; + } + + // Current row is not NULL + // Frame definition has at least one of: X PRECEDING, Y FOLLOWING + int frameStart; + switch (frameInfo.getStartType()) { + case UNBOUNDED_PRECEDING: + frameStart = 0; + break; + case PRECEDING: + frameStart = getPrecedingOffset(currentPosition, peerGroupStart, peerGroupEnd, true); + break; + case CURRENT_ROW: + frameStart = peerGroupStart; + break; + case FOLLOWING: + frameStart = getFollowingOffset(currentPosition, peerGroupStart, peerGroupEnd, true); + break; + default: + // UNBOUND_FOLLOWING is not allowed in frame start + throw new SemanticException("UNBOUND PRECEDING is not allowed in frame start!"); + } + + int frameEnd; + switch (frameInfo.getEndType()) { + case PRECEDING: + frameEnd = getPrecedingOffset(currentPosition, peerGroupStart, peerGroupEnd, false); + break; + case CURRENT_ROW: + frameEnd = peerGroupEnd - 1; + break; + case FOLLOWING: + frameEnd = getFollowingOffset(currentPosition, peerGroupStart, peerGroupEnd, false); + break; + case UNBOUNDED_FOLLOWING: + frameEnd = partitionSize - 1; + break; + default: + // UNBOUND_PRECEDING is not allowed in frame start + throw new SemanticException("UNBOUND PRECEDING is not allowed in frame end!"); + } + + if (frameEnd < frameStart || frameEnd < 0 || frameStart >= partitionSize) { + recentRange = new Range(Math.min(partitionSize - 1, frameStart), Math.max(0, frameEnd)); + return new Range(-1, -1); + } + + frameStart = Math.max(frameStart, 0); + frameEnd = Math.min(frameEnd, partitionSize - 1); + recentRange = new Range(frameStart, frameEnd); + return recentRange; + } + + private int getPrecedingOffset(int index, int peerGroupStart, int peerGroupEnd, boolean isStart) { + int offset; + if (isStart) { + if (!dataType.isNumeric() + && dataType != TSDataType.DATE + && dataType != TSDataType.TIMESTAMP) { + return peerGroupStart; + } + + int recentStart = recentRange.getStart(); + + // Recent start from NULL + // Which means current row is the first non-null row + if (frameInfo.getSortOrder().isNullsFirst() && column.isNull(recentStart)) { + // Then the frame starts with current row + return index; + } + + if (frameInfo.getSortOrder().isAscending()) { + offset = getAscFrameStartPreceding(index, recentStart); + } else { + offset = getDescFrameStartPreceding(index, recentStart); + } + } else { + if (!dataType.isNumeric() + && dataType != TSDataType.DATE + && dataType != TSDataType.TIMESTAMP) { + return peerGroupEnd; + } + + int recentEnd = recentRange.getEnd(); + + // Leave section of leading nulls + if (frameInfo.getSortOrder().isNullsFirst()) { + while (recentEnd < partitionSize && column.isNull(recentEnd)) { + recentEnd++; + } + } + + if (frameInfo.getSortOrder().isAscending()) { + offset = getAscFrameEndPreceding(index, recentEnd); + } else { + offset = getDescFrameEndPreceding(index, recentEnd); + } + } + + return offset; + } + + private int getFollowingOffset(int index, int peerGroupStart, int peerGroupEnd, boolean isStart) { + int offset; + if (isStart) { + if (!dataType.isNumeric() + && dataType != TSDataType.DATE + && dataType != TSDataType.TIMESTAMP) { + return peerGroupStart; + } + + int recentStart = recentRange.getStart(); + + // Leave section of leading nulls + if (recentStart == 0 && frameInfo.getSortOrder().isNullsFirst() && column.isNull(0)) { + // Then the frame starts with current row + recentStart = index; + } + + // Leave section of tailing nulls + if (!frameInfo.getSortOrder().isNullsFirst()) { + while (recentStart >= 0 && column.isNull(recentStart)) { + recentStart--; + } + if (recentStart < 0) { + return recentStart; + } + } + + if (frameInfo.getSortOrder().isAscending()) { + offset = getAscFrameStartFollowing(index, recentStart); + } else { + offset = getDescFrameStartFollowing(index, recentStart); + } + } else { + if (!dataType.isNumeric() + && dataType != TSDataType.DATE + && dataType != TSDataType.TIMESTAMP) { + return peerGroupEnd; + } + + int recentEnd = recentRange.getEnd(); + + // Leave section of leading nulls + if (frameInfo.getSortOrder().isNullsFirst() && column.isNull(recentEnd)) { + // Then the frame starts with current row + recentEnd = index; + } + + if (frameInfo.getSortOrder().isAscending()) { + offset = getAscFrameEndFollowing(index, recentEnd); + } else { + offset = getDescFrameEndFollowing(index, recentEnd); + } + } + + return offset; + } + + // Find first row which satisfy: + // follow >= current + offset + // And stop right there + private int getAscFrameStartFollowing(int currentIndex, int recentIndex) { + while (recentIndex < partitionSize && !column.isNull(recentIndex)) { + if (compareInAscFrameStartFollowing( + currentIndex, recentIndex, frameInfo.getStartOffsetChannel())) { + return recentIndex; + } + recentIndex++; + } + return recentIndex; + } + + private boolean compareInAscFrameStartFollowing(int currentIndex, int recentIndex, int channel) { + checkArgument(!partition.isNull(channel, currentIndex)); + switch (column.getDataType()) { + case INT32: + case DATE: + int currentInt = column.getInt(currentIndex); + int followInt = column.getInt(recentIndex); + int deltaInt = partition.getInt(channel, currentIndex); + return followInt >= currentInt + deltaInt; + case INT64: + case TIMESTAMP: + long currentLong = column.getLong(currentIndex); + long followLong = column.getLong(recentIndex); + long deltaLong = partition.getLong(channel, currentIndex); + return followLong >= currentLong + deltaLong; + case FLOAT: + float currentFloat = column.getFloat(currentIndex); + float followFloat = column.getFloat(recentIndex); + float deltaFloat = partition.getFloat(channel, currentIndex); + return followFloat >= currentFloat + deltaFloat; + case DOUBLE: + double currentDouble = column.getDouble(currentIndex); + double followDouble = column.getDouble(recentIndex); + double deltaDouble = partition.getDouble(channel, currentIndex); + return followDouble >= currentDouble + deltaDouble; + default: + // Unreachable + throw new UnSupportedDataTypeException("Unsupported data type: " + column.getDataType()); + } + } + + // Find first row which satisfy: + // follow > current + offset + // And return its previous index + private int getAscFrameEndFollowing(int currentIndex, int recentIndex) { + while (recentIndex < partitionSize && !column.isNull(recentIndex)) { + if (compareInAscFrameEndFollowing( + currentIndex, recentIndex, frameInfo.getEndOffsetChannel())) { + return recentIndex - 1; + } + recentIndex++; + } + return recentIndex - 1; + } + + private boolean compareInAscFrameEndFollowing(int currentIndex, int recentIndex, int channel) { + checkArgument(!partition.isNull(channel, currentIndex)); + switch (column.getDataType()) { + case INT32: + case DATE: + int currentInt = column.getInt(currentIndex); + int followInt = column.getInt(recentIndex); + int deltaInt = partition.getInt(channel, currentIndex); + return followInt > currentInt + deltaInt; + case INT64: + case TIMESTAMP: + long currentLong = column.getLong(currentIndex); + long followLong = column.getLong(recentIndex); + long deltaLong = partition.getLong(channel, currentIndex); + return followLong > currentLong + deltaLong; + case FLOAT: + float currentFloat = column.getFloat(currentIndex); + float followFloat = column.getFloat(recentIndex); + float deltaFloat = partition.getFloat(channel, currentIndex); + return followFloat > currentFloat + deltaFloat; + case DOUBLE: + double currentDouble = column.getDouble(currentIndex); + double followDouble = column.getDouble(recentIndex); + double deltaDouble = partition.getDouble(channel, currentIndex); + return followDouble > currentDouble + deltaDouble; + default: + // Unreachable + throw new UnSupportedDataTypeException("Unsupported data type: " + column.getDataType()); + } + } + + // Find first row which satisfy: + // precede >= current - offset + // And stop right there + private int getAscFrameStartPreceding(int currentIndex, int recentIndex) { + while (recentIndex < currentIndex) { + if (compareInAscFrameStartPreceding( + currentIndex, recentIndex, frameInfo.getStartOffsetChannel())) { + return recentIndex; + } + recentIndex++; + } + return recentIndex; + } + + private boolean compareInAscFrameStartPreceding(int currentIndex, int recentIndex, int channel) { + checkArgument(!partition.isNull(channel, currentIndex)); + switch (column.getDataType()) { + case INT32: + case DATE: + int currentInt = column.getInt(currentIndex); + int precedeInt = column.getInt(recentIndex); + int deltaInt = partition.getInt(channel, currentIndex); + return precedeInt >= currentInt - deltaInt; + case INT64: + case TIMESTAMP: + long currentLong = column.getLong(currentIndex); + long precedeLong = column.getLong(recentIndex); + long deltaLong = partition.getLong(channel, currentIndex); + return precedeLong >= currentLong - deltaLong; + case FLOAT: + float currentFloat = column.getFloat(currentIndex); + float precedeFollow = column.getFloat(recentIndex); + float deltaFloat = partition.getFloat(channel, currentIndex); + return precedeFollow >= currentFloat - deltaFloat; + case DOUBLE: + double currentDouble = column.getDouble(currentIndex); + double precedeDouble = column.getDouble(recentIndex); + double deltaDouble = partition.getDouble(channel, currentIndex); + return precedeDouble >= currentDouble - deltaDouble; + default: + // Unreachable + throw new UnSupportedDataTypeException("Unsupported data type: " + column.getDataType()); + } + } + + // Find first row which satisfy: + // precede > current - offset + // And return its previous index + private int getAscFrameEndPreceding(int currentIndex, int recentIndex) { + while (recentIndex < partitionSize) { + if (compareInAscFrameEndPreceding( + currentIndex, recentIndex, frameInfo.getEndOffsetChannel())) { + return recentIndex - 1; + } + recentIndex++; + } + return recentIndex - 1; + } + + private boolean compareInAscFrameEndPreceding(int currentIndex, int recentIndex, int channel) { + checkArgument(!partition.isNull(channel, currentIndex)); + switch (column.getDataType()) { + case INT32: + case DATE: + int currentInt = column.getInt(currentIndex); + int precedeInt = column.getInt(recentIndex); + int deltaInt = partition.getInt(channel, currentIndex); + return precedeInt > currentInt - deltaInt; + case INT64: + case TIMESTAMP: + long currentLong = column.getLong(currentIndex); + long precedeLong = column.getLong(recentIndex); + long deltaLong = partition.getLong(channel, currentIndex); + return precedeLong > currentLong - deltaLong; + case FLOAT: + float currentFloat = column.getFloat(currentIndex); + float precedeFollow = column.getFloat(recentIndex); + float deltaFloat = partition.getFloat(channel, currentIndex); + return precedeFollow > currentFloat - deltaFloat; + case DOUBLE: + double currentDouble = column.getDouble(currentIndex); + double precedeDouble = column.getDouble(recentIndex); + double deltaDouble = partition.getDouble(channel, currentIndex); + return precedeDouble > currentDouble - deltaDouble; + default: + // Unreachable + throw new UnSupportedDataTypeException("Unsupported data type: " + column.getDataType()); + } + } + + // Find first row which satisfy: + // follow <= current - offset + // And stop right there + private int getDescFrameStartFollowing(int currentIndex, int recentIndex) { + while (recentIndex < partitionSize && !column.isNull(recentIndex)) { + if (compareInDescFrameStartFollowing( + currentIndex, recentIndex, frameInfo.getStartOffsetChannel())) { + return recentIndex; + } + recentIndex++; + } + return recentIndex; + } + + private boolean compareInDescFrameStartFollowing(int currentIndex, int recentIndex, int channel) { + checkArgument(!partition.isNull(channel, currentIndex)); + switch (column.getDataType()) { + case INT32: + case DATE: + int currentInt = column.getInt(currentIndex); + int followInt = column.getInt(recentIndex); + int deltaInt = partition.getInt(channel, currentIndex); + return followInt <= currentInt - deltaInt; + case INT64: + case TIMESTAMP: + long currentLong = column.getLong(currentIndex); + long followLong = column.getLong(recentIndex); + long deltaLong = partition.getLong(channel, currentIndex); + return followLong <= currentLong - deltaLong; + case FLOAT: + float currentFloat = column.getFloat(currentIndex); + float followFloat = column.getFloat(recentIndex); + float deltaFloat = partition.getFloat(channel, currentIndex); + return followFloat <= currentFloat - deltaFloat; + case DOUBLE: + double currentDouble = column.getDouble(currentIndex); + double followDouble = column.getDouble(recentIndex); + double deltaDouble = partition.getDouble(channel, currentIndex); + return followDouble <= currentDouble - deltaDouble; + default: + // Unreachable + throw new UnSupportedDataTypeException("Unsupported data type: " + column.getDataType()); + } + } + + // Find first row which satisfy: + // follow < current - offset + // And return its previous index + private int getDescFrameEndFollowing(int currentIndex, int recentIndex) { + while (recentIndex < partitionSize && !column.isNull(recentIndex)) { + if (compareInDescFrameEndFollowing( + currentIndex, recentIndex, frameInfo.getEndOffsetChannel())) { + return recentIndex - 1; + } + recentIndex++; + } + return recentIndex - 1; + } + + private boolean compareInDescFrameEndFollowing(int currentIndex, int recentIndex, int channel) { + checkArgument(!partition.isNull(channel, currentIndex)); + switch (column.getDataType()) { + case INT32: + case DATE: + int currentInt = column.getInt(currentIndex); + int followInt = column.getInt(recentIndex); + int deltaInt = partition.getInt(channel, currentIndex); + return followInt < currentInt - deltaInt; + case INT64: + case TIMESTAMP: + long currentLong = column.getLong(currentIndex); + long followLong = column.getLong(recentIndex); + long deltaLong = partition.getLong(channel, currentIndex); + return followLong < currentLong - deltaLong; + case FLOAT: + float currentFloat = column.getFloat(currentIndex); + float followFloat = column.getFloat(recentIndex); + float deltaFloat = partition.getFloat(channel, currentIndex); + return followFloat < currentFloat - deltaFloat; + case DOUBLE: + double currentDouble = column.getDouble(currentIndex); + double followDouble = column.getDouble(recentIndex); + double deltaDouble = partition.getDouble(channel, currentIndex); + return followDouble < currentDouble - deltaDouble; + default: + // Unreachable + throw new UnSupportedDataTypeException("Unsupported data type: " + column.getDataType()); + } + } + + // Find first row which satisfy: + // precede <= current + offset + // And stop right there + private int getDescFrameStartPreceding(int currentIndex, int recentIndex) { + while (recentIndex < currentIndex) { + if (compareInDescFrameStartPreceding( + currentIndex, recentIndex, frameInfo.getStartOffsetChannel())) { + return recentIndex; + } + recentIndex++; + } + return recentIndex; + } + + private boolean compareInDescFrameStartPreceding(int currentIndex, int recentIndex, int channel) { + checkArgument(!partition.isNull(channel, currentIndex)); + switch (column.getDataType()) { + case INT32: + case DATE: + int currentInt = column.getInt(currentIndex); + int precedeInt = column.getInt(recentIndex); + int deltaInt = partition.getInt(channel, currentIndex); + return precedeInt <= currentInt + deltaInt; + case INT64: + case TIMESTAMP: + long currentLong = column.getLong(currentIndex); + long precedeLong = column.getLong(recentIndex); + long deltaLong = partition.getLong(channel, currentIndex); + return precedeLong <= currentLong + deltaLong; + case FLOAT: + float currentFloat = column.getFloat(currentIndex); + float precedeFollow = column.getFloat(recentIndex); + float deltaFloat = partition.getFloat(channel, currentIndex); + return precedeFollow <= currentFloat + deltaFloat; + case DOUBLE: + double currentDouble = column.getDouble(currentIndex); + double precedeDouble = column.getDouble(recentIndex); + double deltaDouble = partition.getDouble(channel, currentIndex); + return precedeDouble <= currentDouble + deltaDouble; + default: + // Unreachable + throw new UnSupportedDataTypeException("Unsupported data type: " + column.getDataType()); + } + } + + // Find first row which satisfy: + // precede < current + offset + // And return its previous index + private int getDescFrameEndPreceding(int currentIndex, int recentIndex) { + while (recentIndex < partitionSize) { + if (compareInDescFrameEndPreceding( + currentIndex, recentIndex, frameInfo.getEndOffsetChannel())) { + return recentIndex - 1; + } + recentIndex++; + } + return recentIndex - 1; + } + + private boolean compareInDescFrameEndPreceding(int currentIndex, int recentIndex, int channel) { + checkArgument(!partition.isNull(channel, currentIndex)); + switch (column.getDataType()) { + case INT32: + case DATE: + int currentInt = column.getInt(currentIndex); + int precedeInt = column.getInt(recentIndex); + int deltaInt = partition.getInt(channel, currentIndex); + return precedeInt < currentInt + deltaInt; + case INT64: + case TIMESTAMP: + long currentLong = column.getLong(currentIndex); + long precedeLong = column.getLong(recentIndex); + long deltaLong = partition.getLong(channel, currentIndex); + return precedeLong < currentLong + deltaLong; + case FLOAT: + float currentFloat = column.getFloat(currentIndex); + float precedeFollow = column.getFloat(recentIndex); + float deltaFloat = partition.getFloat(channel, currentIndex); + return precedeFollow < currentFloat + deltaFloat; + case DOUBLE: + double currentDouble = column.getDouble(currentIndex); + double precedeDouble = column.getDouble(recentIndex); + double deltaDouble = partition.getDouble(channel, currentIndex); + return precedeDouble < currentDouble + deltaDouble; + default: + // Unreachable + throw new UnSupportedDataTypeException("Unsupported data type: " + column.getDataType()); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RowsFrame.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RowsFrame.java new file mode 100644 index 000000000000..2177bb68c8e3 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RowsFrame.java @@ -0,0 +1,108 @@ +/* + * 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.process.window.partition.frame; + +import org.apache.iotdb.db.exception.sql.SemanticException; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.Range; + +import static com.google.common.base.Preconditions.checkArgument; + +public class RowsFrame implements Frame { + private final Partition partition; + private final FrameInfo frameInfo; + private final int partitionStart; + private final int partitionSize; + + public RowsFrame(Partition partition, FrameInfo frameInfo, int partitionStart, int partitionEnd) { + checkArgument(frameInfo.getFrameType() == FrameInfo.FrameType.ROWS); + + this.partition = partition; + this.frameInfo = frameInfo; + this.partitionStart = partitionStart; + this.partitionSize = partitionEnd - partitionStart; + } + + @Override + public Range getRange( + int currentPosition, int currentGroup, int peerGroupStart, int peerGroupEnd) { + int posInPartition = currentPosition - partitionStart; + + int offset; + int frameStart; + switch (frameInfo.getStartType()) { + case UNBOUNDED_PRECEDING: + frameStart = 0; + break; + case PRECEDING: + offset = (int) getOffset(frameInfo.getStartOffsetChannel(), currentPosition); + frameStart = posInPartition - offset; + break; + case CURRENT_ROW: + frameStart = posInPartition; + break; + case FOLLOWING: + offset = (int) getOffset(frameInfo.getStartOffsetChannel(), currentPosition); + frameStart = posInPartition + offset; + break; + default: + // UNBOUND_FOLLOWING is not allowed in frame start + throw new SemanticException("UNBOUND PRECEDING is not allowed in frame start!"); + } + + int frameEnd; + switch (frameInfo.getEndType()) { + case PRECEDING: + offset = (int) getOffset(frameInfo.getEndOffsetChannel(), currentPosition); + frameEnd = posInPartition - offset; + break; + case CURRENT_ROW: + frameEnd = posInPartition; + break; + case FOLLOWING: + offset = (int) getOffset(frameInfo.getEndOffsetChannel(), currentPosition); + frameEnd = posInPartition + offset; + break; + case UNBOUNDED_FOLLOWING: + frameEnd = partitionSize - 1; + break; + default: + // UNBOUND_PRECEDING is not allowed in frame end + throw new SemanticException("UNBOUND PRECEDING is not allowed in frame end!"); + } + + // Empty frame + if (frameEnd < frameStart || frameEnd < 0 || frameStart >= partitionSize) { + return new Range(-1, -1); + } + + frameStart = Math.max(frameStart, 0); + frameEnd = Math.min(frameEnd, partitionSize - 1); + return new Range(frameStart, frameEnd); + } + + public long getOffset(int channel, int index) { + checkArgument(!partition.isNull(channel, index)); + long offset = partition.getLong(channel, index); + + checkArgument(offset >= 0); + return offset; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/ColumnList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/ColumnList.java new file mode 100644 index 000000000000..97152b715d80 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/ColumnList.java @@ -0,0 +1,141 @@ +/* + * 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.process.window.utils; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.block.column.ColumnEncoding; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.utils.Binary; + +import java.util.ArrayList; +import java.util.List; + +public class ColumnList { + private final List columns; + private final List positionCounts; + + public ColumnList(List columns) { + this.columns = columns; + + positionCounts = new ArrayList<>(columns.size()); + for (Column column : columns) { + positionCounts.add(column.getPositionCount()); + } + } + + public TSDataType getDataType() { + return columns.get(0).getDataType(); + } + + public ColumnEncoding getEncoding() { + return columns.get(0).getEncoding(); + } + + public static class ColumnListIndex { + private final int columnIndex; + private final int offsetInColumn; + + ColumnListIndex(int columnIndex, int offsetInColumn) { + this.columnIndex = columnIndex; + this.offsetInColumn = offsetInColumn; + } + + public int getColumnIndex() { + return columnIndex; + } + + public int getOffsetInColumn() { + return offsetInColumn; + } + } + + public ColumnListIndex getColumnListIndex(int rowIndex) { + int columnIndex = 0; + while (columnIndex < columns.size() && rowIndex >= positionCounts.get(columnIndex)) { + rowIndex -= positionCounts.get(columnIndex); + // Enter next Column + columnIndex++; + } + + if (columnIndex != columns.size()) { + return new ColumnListIndex(columnIndex, rowIndex); + } else { + // Unlikely + throw new IndexOutOfBoundsException("Index out of Partition's bounds!"); + } + } + + public boolean getBoolean(int position) { + ColumnListIndex columnListIndex = getColumnListIndex(position); + int columnIndex = columnListIndex.getColumnIndex(); + int offsetInColumn = columnListIndex.getOffsetInColumn(); + + return columns.get(columnIndex).getBoolean(offsetInColumn); + } + + public int getInt(int position) { + ColumnListIndex columnListIndex = getColumnListIndex(position); + int columnIndex = columnListIndex.getColumnIndex(); + int offsetInColumn = columnListIndex.getOffsetInColumn(); + return columns.get(columnIndex).getInt(offsetInColumn); + } + + public long getLong(int position) { + ColumnListIndex columnListIndex = getColumnListIndex(position); + int columnIndex = columnListIndex.getColumnIndex(); + int offsetInColumn = columnListIndex.getOffsetInColumn(); + return columns.get(columnIndex).getLong(offsetInColumn); + } + + public float getFloat(int position) { + ColumnListIndex columnListIndex = getColumnListIndex(position); + int columnIndex = columnListIndex.getColumnIndex(); + int offsetInColumn = columnListIndex.getOffsetInColumn(); + return columns.get(columnIndex).getFloat(offsetInColumn); + } + + public double getDouble(int position) { + ColumnListIndex columnListIndex = getColumnListIndex(position); + int columnIndex = columnListIndex.getColumnIndex(); + int offsetInColumn = columnListIndex.getOffsetInColumn(); + return columns.get(columnIndex).getDouble(offsetInColumn); + } + + public Binary getBinary(int position) { + ColumnListIndex columnListIndex = getColumnListIndex(position); + int columnIndex = columnListIndex.getColumnIndex(); + int offsetInColumn = columnListIndex.getOffsetInColumn(); + return columns.get(columnIndex).getBinary(offsetInColumn); + } + + public Object getObject(int position) { + ColumnListIndex columnListIndex = getColumnListIndex(position); + int columnIndex = columnListIndex.getColumnIndex(); + int offsetInColumn = columnListIndex.getOffsetInColumn(); + return columns.get(columnIndex).getObject(offsetInColumn); + } + + public boolean isNull(int position) { + ColumnListIndex columnListIndex = getColumnListIndex(position); + int columnIndex = columnListIndex.getColumnIndex(); + int offsetInColumn = columnListIndex.getOffsetInColumn(); + return columns.get(columnIndex).isNull(offsetInColumn); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/Range.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/Range.java new file mode 100644 index 000000000000..9994742bbfee --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/Range.java @@ -0,0 +1,38 @@ +/* + * 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.process.window.utils; + +public class Range { + private final int start; + private final int end; + + public Range(int start, int end) { + this.start = start; + this.end = end; + } + + public int getStart() { + return start; + } + + public int getEnd() { + return end; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/RowComparator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/RowComparator.java new file mode 100644 index 000000000000..768eeaed8542 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/utils/RowComparator.java @@ -0,0 +1,231 @@ +/* + * 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.process.window.utils; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.write.UnSupportedDataTypeException; + +import java.util.List; + +public class RowComparator { + private final List dataTypes; + + public RowComparator(List dataTypes) { + this.dataTypes = dataTypes; + } + + public boolean equalColumns(List columns, int offset1, int offset2) { + for (int i = 0; i < dataTypes.size(); i++) { + Column column = columns.get(i); + TSDataType dataType = dataTypes.get(i); + if (!equal(column, dataType, offset1, offset2)) { + return false; + } + } + return true; + } + + public boolean equal(Column column, int offset1, int offset2) { + assert dataTypes.size() == 1; + return equal(column, dataTypes.get(0), offset1, offset2); + } + + private boolean equal(Column column, TSDataType dataType, int offset1, int offset2) { + switch (dataType) { + case BOOLEAN: + boolean bool1 = column.getBoolean(offset1); + boolean bool2 = column.getBoolean(offset2); + if (bool1 != bool2) { + return false; + } + break; + case INT32: + int int1 = column.getInt(offset1); + int int2 = column.getInt(offset2); + if (int1 != int2) { + return false; + } + break; + case INT64: + long long1 = column.getLong(offset1); + long long2 = column.getLong(offset2); + if (long1 != long2) { + return false; + } + break; + case FLOAT: + float float1 = column.getFloat(offset1); + float float2 = column.getFloat(offset2); + if (float1 != float2) { + return false; + } + break; + case DOUBLE: + double double1 = column.getDouble(offset1); + double double2 = column.getDouble(offset2); + if (double1 != double2) { + return false; + } + break; + case TEXT: + Binary bin1 = column.getBinary(offset1); + Binary bin2 = column.getBinary(offset2); + if (!bin1.equals(bin2)) { + return false; + } + break; + default: + // Would throw at the first run + throw new UnSupportedDataTypeException(dataType.toString()); + } + return true; + } + + public boolean equalColumnLists(List columns, int offset1, int offset2) { + for (int i = 0; i < dataTypes.size(); i++) { + ColumnList column = columns.get(i); + TSDataType dataType = dataTypes.get(i); + if (!equal(column, dataType, offset1, offset2)) { + return false; + } + } + return true; + } + + public boolean equal(ColumnList column, int offset1, int offset2) { + assert dataTypes.size() == 1; + return equal(column, dataTypes.get(0), offset1, offset2); + } + + private boolean equal(ColumnList column, TSDataType dataType, int offset1, int offset2) { + switch (dataType) { + case BOOLEAN: + boolean bool1 = column.getBoolean(offset1); + boolean bool2 = column.getBoolean(offset2); + if (bool1 != bool2) { + return false; + } + break; + case INT32: + int int1 = column.getInt(offset1); + int int2 = column.getInt(offset2); + if (int1 != int2) { + return false; + } + break; + case INT64: + long long1 = column.getLong(offset1); + long long2 = column.getLong(offset2); + if (long1 != long2) { + return false; + } + break; + case FLOAT: + float float1 = column.getFloat(offset1); + float float2 = column.getFloat(offset2); + if (float1 != float2) { + return false; + } + break; + case DOUBLE: + double double1 = column.getDouble(offset1); + double double2 = column.getDouble(offset2); + if (double1 != double2) { + return false; + } + break; + case TEXT: + Binary bin1 = column.getBinary(offset1); + Binary bin2 = column.getBinary(offset2); + if (!bin1.equals(bin2)) { + return false; + } + break; + default: + // Would throw at the first run + throw new UnSupportedDataTypeException(dataType.toString()); + } + return true; + } + + public boolean equal(List columns1, int offset1, List columns2, int offset2) { + for (int i = 0; i < dataTypes.size(); i++) { + TSDataType dataType = dataTypes.get(i); + Column column1 = columns1.get(i); + Column column2 = columns2.get(i); + + switch (dataType) { + case BOOLEAN: + boolean bool1 = column1.getBoolean(offset1); + boolean bool2 = column2.getBoolean(offset2); + if (bool1 != bool2) { + return false; + } + break; + case INT32: + int int1 = column1.getInt(offset1); + int int2 = column2.getInt(offset2); + if (int1 != int2) { + return false; + } + break; + case INT64: + long long1 = column1.getLong(offset1); + long long2 = column2.getLong(offset2); + if (long1 != long2) { + return false; + } + break; + case FLOAT: + float float1 = column1.getFloat(offset1); + float float2 = column2.getFloat(offset2); + if (float1 != float2) { + return false; + } + break; + case DOUBLE: + double double1 = column1.getDouble(offset1); + double double2 = column2.getDouble(offset2); + if (double1 != double2) { + return false; + } + break; + case TEXT: + Binary bin1 = column1.getBinary(offset1); + Binary bin2 = column2.getBinary(offset2); + if (!bin1.equals(bin2)) { + return false; + } + break; + default: + // Would throw at the first run + throw new UnSupportedDataTypeException(dataType.toString()); + } + } + + return true; + } + + public List getDataTypes() { + return dataTypes; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AvgAccumulator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AvgAccumulator.java index b4a97b19e0d1..935a114cbdfd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AvgAccumulator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/aggregation/AvgAccumulator.java @@ -319,7 +319,7 @@ private void removeFloatInput(Column column) { for (int i = 0; i < count; i++) { if (!column.isNull(i)) { countValue--; - sumValue += column.getFloat(i); + sumValue -= column.getFloat(i); } } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/TableWindowOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/TableWindowOperatorTest.java new file mode 100644 index 000000000000..ed0ec405a9c4 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/TableWindowOperatorTest.java @@ -0,0 +1,320 @@ +/* + * 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.process.window; + +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.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.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.execution.operator.process.TreeLinearFillOperator; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.WindowFunction; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.rank.RankFunction; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.FrameInfo; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.apache.tsfile.utils.Binary; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +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.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +public class TableWindowOperatorTest { + private static final ExecutorService instanceNotificationExecutor = + IoTDBThreadPoolFactory.newFixedThreadPool(1, "windowOperator-test-instance-notification"); + + private final long[] column1 = new long[] {1, 2, 3, 4, 5, 6, 7}; + private final String[] column2 = new String[] {"d1", "d1", "d2", "d2", "d2", "d2", "d2"}; + private final int[] column3 = new int[] {1, 2, 3, 4, 5, 6, 7}; + private final long[] column4 = new long[] {1, 2, 1, 2, 3, 4, 5}; + + @Test + public void testOneTsBlockWithMultiPartition() { + long[][] timeArray = + new long[][] { + {1, 2, 3, 4, 5, 6, 7}, + }; + String[][] deviceIdArray = + new String[][] { + {"d1", "d1", "d2", "d2", "d2", "d2", "d2"}, + }; + int[][] valueArray = + new int[][] { + {1, 2, 3, 4, 5, 6, 7}, + }; + + int count = 0; + try (TableWindowOperator windowOperator = + genWindowOperator(timeArray, deviceIdArray, valueArray)) { + ListenableFuture listenableFuture = windowOperator.isBlocked(); + listenableFuture.get(); + while (!windowOperator.isFinished() && windowOperator.hasNext()) { + TsBlock tsBlock = windowOperator.next(); + if (tsBlock != null && !tsBlock.isEmpty()) { + for (int i = 0, size = tsBlock.getPositionCount(); i < size; i++, count++) { + assertEquals(column1[count], tsBlock.getColumn(0).getLong(i)); + assertEquals( + column2[count], + tsBlock.getColumn(1).getBinary(i).getStringValue(TSFileConfig.STRING_CHARSET)); + assertEquals(column3[count], tsBlock.getColumn(2).getInt(i)); + assertEquals(column4[count], tsBlock.getColumn(3).getLong(i)); + } + } + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testPartitionCrossMultiTsBlock() { + long[][] timeArray = + new long[][] { + {1, 2}, + {3, 4}, + {5}, + {6, 7}, + }; + String[][] deviceIdArray = + new String[][] { + {"d1", "d1"}, + {"d2", "d2"}, + {"d2"}, + {"d2", "d2"}, + }; + int[][] valueArray = + new int[][] { + {1, 2}, + {3, 4}, + {5}, + {6, 7}, + }; + + int count = 0; + try (TableWindowOperator windowOperator = + genWindowOperator(timeArray, deviceIdArray, valueArray)) { + ListenableFuture listenableFuture = windowOperator.isBlocked(); + listenableFuture.get(); + while (!windowOperator.isFinished() && windowOperator.hasNext()) { + TsBlock tsBlock = windowOperator.next(); + if (tsBlock != null && !tsBlock.isEmpty()) { + for (int i = 0, size = tsBlock.getPositionCount(); i < size; i++, count++) { + assertEquals(column1[count], tsBlock.getColumn(0).getLong(i)); + assertEquals( + column2[count], + tsBlock.getColumn(1).getBinary(i).getStringValue(TSFileConfig.STRING_CHARSET)); + assertEquals(column3[count], tsBlock.getColumn(2).getInt(i)); + assertEquals(column4[count], tsBlock.getColumn(3).getLong(i)); + } + } + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testMixedPartition() { + long[][] timeArray = + new long[][] { + {1, 2, 3, 4}, + {5, 6, 7}, + }; + String[][] deviceIdArray = + new String[][] { + {"d1", "d1", "d2", "d2"}, + {"d2", "d2", "d2"}, + }; + int[][] valueArray = + new int[][] { + {1, 2, 3, 4}, + {5, 6, 7}, + }; + + int count = 0; + try (TableWindowOperator windowOperator = + genWindowOperator(timeArray, deviceIdArray, valueArray)) { + ListenableFuture listenableFuture = windowOperator.isBlocked(); + listenableFuture.get(); + while (!windowOperator.isFinished() && windowOperator.hasNext()) { + TsBlock tsBlock = windowOperator.next(); + if (tsBlock != null && !tsBlock.isEmpty()) { + for (int i = 0, size = tsBlock.getPositionCount(); i < size; i++, count++) { + assertEquals(column1[count], tsBlock.getColumn(0).getLong(i)); + assertEquals( + column2[count], + tsBlock.getColumn(1).getBinary(i).getStringValue(TSFileConfig.STRING_CHARSET)); + assertEquals(column3[count], tsBlock.getColumn(2).getInt(i)); + assertEquals(column4[count], tsBlock.getColumn(3).getLong(i)); + } + } + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + static class ChildOperator implements Operator { + private int index; + + private final long[][] timeArray; + private final String[][] deviceIdArray; + private final int[][] valueArray; + private final DriverContext driverContext; + + ChildOperator( + long[][] timeArray, + String[][] deviceIdArray, + int[][] valueArray, + DriverContext driverContext) { + this.timeArray = timeArray; + this.deviceIdArray = deviceIdArray; + this.valueArray = valueArray; + this.driverContext = driverContext; + + this.index = 0; + } + + @Override + public OperatorContext getOperatorContext() { + return driverContext.getOperatorContexts().get(0); + } + + @Override + public TsBlock next() throws Exception { + if (timeArray[index] == null) { + index++; + return null; + } + TsBlockBuilder builder = + new TsBlockBuilder( + timeArray[index].length, + Arrays.asList(TSDataType.TIMESTAMP, TSDataType.TEXT, TSDataType.INT32)); + for (int i = 0, size = timeArray[index].length; i < size; i++) { + builder.getColumnBuilder(0).writeLong(timeArray[index][i]); + builder + .getColumnBuilder(1) + .writeBinary(new Binary(deviceIdArray[index][i], TSFileConfig.STRING_CHARSET)); + builder.getColumnBuilder(2).writeInt(valueArray[index][i]); + } + builder.declarePositions(timeArray[index].length); + index++; + return builder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, builder.getPositionCount())); + } + + @Override + public boolean hasNext() { + return index < timeArray.length; + } + + @Override + public boolean isFinished() { + return index >= timeArray.length; + } + + @Override + public void close() { + // do nothing + } + + @Override + public long calculateMaxPeekMemory() { + return 0; + } + + @Override + public long calculateMaxReturnSize() { + return 0; + } + + @Override + public long calculateRetainedSizeAfterCallingNext() { + return 0; + } + + @Override + public long ramBytesUsed() { + return 0; + } + } + + private TableWindowOperator genWindowOperator( + long[][] timeArray, String[][] deviceIdArray, int[][] valueArray) { + QueryId queryId = new QueryId("stub_query"); + FragmentInstanceId instanceId = + new FragmentInstanceId(new PlanFragmentId(queryId, 0), "stub-instance"); + FragmentInstanceStateMachine stateMachine = + new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor); + FragmentInstanceContext fragmentInstanceContext = + createFragmentInstanceContext(instanceId, stateMachine); + DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0); + PlanNodeId planNode = new PlanNodeId("1"); + driverContext.addOperatorContext(1, planNode, TreeLinearFillOperator.class.getSimpleName()); + + List inputDataTypes = + Arrays.asList(TSDataType.TIMESTAMP, TSDataType.TEXT, TSDataType.INT32); + List outputDataTypes = + Arrays.asList(TSDataType.TIMESTAMP, TSDataType.TEXT, TSDataType.INT32, TSDataType.INT64); + ArrayList outputChannels = new ArrayList<>(); + for (int i = 0; i < inputDataTypes.size(); i++) { + outputChannels.add(i); + } + WindowFunction windowFunction = new RankFunction(); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.CURRENT_ROW); + + Operator childOperator = new ChildOperator(timeArray, deviceIdArray, valueArray, driverContext); + return new TableWindowOperator( + driverContext.getOperatorContexts().get(0), + childOperator, + inputDataTypes, + outputDataTypes, + outputChannels, + Collections.singletonList(windowFunction), + Collections.singletonList(frameInfo), + Collections.singletonList(1), + Collections.singletonList(2)); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/TableWindowOperatorTestUtils.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/TableWindowOperatorTestUtils.java new file mode 100644 index 000000000000..3cdaa0fdb1f6 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/TableWindowOperatorTestUtils.java @@ -0,0 +1,131 @@ +/* + * 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.process.window; + +import org.apache.tsfile.block.column.ColumnBuilder; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class TableWindowOperatorTestUtils { + public static TsBlock createIntsTsBlockWithoutNulls(int[] inputs) { + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(Collections.singletonList(TSDataType.INT32)); + ColumnBuilder[] columnBuilders = tsBlockBuilder.getValueColumnBuilders(); + for (int input : inputs) { + columnBuilders[0].writeInt(input); + tsBlockBuilder.declarePosition(); + } + + return tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + } + + public static TsBlock createIntsTsBlockWithoutNulls(int[] inputs, int offset) { + TsBlockBuilder tsBlockBuilder = + new TsBlockBuilder(Arrays.asList(TSDataType.INT32, TSDataType.INT32)); + ColumnBuilder[] columnBuilders = tsBlockBuilder.getValueColumnBuilders(); + for (int input : inputs) { + columnBuilders[0].writeInt(input); + columnBuilders[1].writeInt(offset); + tsBlockBuilder.declarePosition(); + } + + return tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + } + + public static TsBlock createIntsTsBlockWithoutNulls( + int[] inputs, int startOffset, int endOffset) { + TsBlockBuilder tsBlockBuilder = + new TsBlockBuilder(Arrays.asList(TSDataType.INT32, TSDataType.INT32, TSDataType.INT32)); + ColumnBuilder[] columnBuilders = tsBlockBuilder.getValueColumnBuilders(); + for (int input : inputs) { + columnBuilders[0].writeInt(input); + columnBuilders[1].writeInt(startOffset); + columnBuilders[2].writeInt(endOffset); + tsBlockBuilder.declarePosition(); + } + + return tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + } + + public static TsBlock createIntsTsBlockWithNulls(int[] inputs) { + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(Collections.singletonList(TSDataType.INT32)); + ColumnBuilder[] columnBuilders = tsBlockBuilder.getValueColumnBuilders(); + for (int input : inputs) { + if (input >= 0) { + columnBuilders[0].writeInt(input); + } else { + // Mimic null value + columnBuilders[0].appendNull(); + } + tsBlockBuilder.declarePosition(); + } + + return tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + } + + public static TsBlock createIntsTsBlockWithNulls(int[] inputs, int offset) { + TsBlockBuilder tsBlockBuilder = + new TsBlockBuilder(Arrays.asList(TSDataType.INT32, TSDataType.INT32)); + ColumnBuilder[] columnBuilders = tsBlockBuilder.getValueColumnBuilders(); + for (int input : inputs) { + if (input >= 0) { + columnBuilders[0].writeInt(input); + } else { + // Mimic null value + columnBuilders[0].appendNull(); + } + columnBuilders[1].writeInt(offset); + tsBlockBuilder.declarePosition(); + } + + return tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + } + + public static TsBlock createIntsTsBlockWithNulls(int[] inputs, int startOffset, int endOffset) { + TsBlockBuilder tsBlockBuilder = + new TsBlockBuilder(Arrays.asList(TSDataType.INT32, TSDataType.INT32, TSDataType.INT32)); + ColumnBuilder[] columnBuilders = tsBlockBuilder.getValueColumnBuilders(); + for (int input : inputs) { + if (input >= 0) { + columnBuilders[0].writeInt(input); + } else { + // Mimic null value + columnBuilders[0].appendNull(); + } + columnBuilders[1].writeInt(startOffset); + columnBuilders[2].writeInt(endOffset); + tsBlockBuilder.declarePosition(); + } + + return tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/FunctionTestUtils.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/FunctionTestUtils.java new file mode 100644 index 000000000000..ccec0f68563b --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/FunctionTestUtils.java @@ -0,0 +1,113 @@ +/* + * 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.process.window.function; + +import org.apache.iotdb.common.rpc.thrift.TAggregationType; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.aggregate.AggregationWindowFunction; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.aggregate.WindowAggregator; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.FrameInfo; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.AccumulatorFactory; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.TableAccumulator; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; + +public class FunctionTestUtils { + public static PartitionExecutor createPartitionExecutor( + TsBlock tsBlock, List dataTypes, WindowFunction windowFunction) { + return createPartitionExecutor(tsBlock, dataTypes, windowFunction, new ArrayList<>()); + } + + public static PartitionExecutor createPartitionExecutor( + TsBlock tsBlock, + List dataTypes, + WindowFunction windowFunction, + List sortChannels) { + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.CURRENT_ROW); + return createPartitionExecutor(tsBlock, dataTypes, windowFunction, frameInfo, sortChannels); + } + + public static PartitionExecutor createPartitionExecutor( + TsBlock tsBlock, + List dataTypes, + WindowFunction windowFunction, + FrameInfo frameInfo) { + return createPartitionExecutor( + tsBlock, dataTypes, windowFunction, frameInfo, new ArrayList<>()); + } + + public static PartitionExecutor createPartitionExecutor( + TsBlock tsBlock, + List dataTypes, + WindowFunction windowFunction, + FrameInfo frameInfo, + List sortChannels) { + List tsBlocks = Collections.singletonList(tsBlock); + int startIndex = 0, endIndex = tsBlock.getPositionCount(); + List windowFunctions = Collections.singletonList(windowFunction); + List frameInfoList = Collections.singletonList(frameInfo); + + // Output channels are contiguous + ArrayList outputChannels = new ArrayList<>(); + for (int i = 0; i < dataTypes.size(); i++) { + outputChannels.add(i); + } + + return new PartitionExecutor( + tsBlocks, + dataTypes, + startIndex, + endIndex, + outputChannels, + windowFunctions, + frameInfoList, + sortChannels); + } + + // Assume input TsBlock has only one column + // And only output one column + public static AggregationWindowFunction createAggregationWindowFunction( + TAggregationType aggregationType, + TSDataType inputDataType, + TSDataType outputDataType, + boolean ascending) { + // inputExpressions and inputAttributes are not used in this method + TableAccumulator accumulator = + AccumulatorFactory.createBuiltinAccumulator( + aggregationType, + Collections.singletonList(inputDataType), + new ArrayList<>(), + new HashMap<>(), + ascending); + WindowAggregator aggregator = + new WindowAggregator(accumulator, outputDataType, Collections.singletonList(0)); + return new AggregationWindowFunction(aggregator); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/aggregate/AggregationWindowFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/aggregate/AggregationWindowFunctionTest.java new file mode 100644 index 000000000000..91a60124cbdf --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/aggregate/AggregationWindowFunctionTest.java @@ -0,0 +1,195 @@ +/* + * 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.process.window.function.aggregate; + +import org.apache.iotdb.common.rpc.thrift.TAggregationType; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.FrameInfo; +import org.apache.iotdb.db.queryengine.plan.relational.planner.SortOrder; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +// For aggregator that supports removeInputs, only SUM is tested, others are similar +public class AggregationWindowFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + private final int[] inputs = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}; + + @Test + public void testFrameExpansion() { + List outputDataTypes = Arrays.asList(TSDataType.INT32, TSDataType.DOUBLE); + double[] expected = {0, 1, 3, 6, 10, 15, 21, 28, 36, 45}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + AggregationWindowFunction function = + FunctionTestUtils.createAggregationWindowFunction( + TAggregationType.SUM, TSDataType.INT32, TSDataType.DOUBLE, true); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.CURRENT_ROW); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, frameInfo); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + // This floating point are integers, no delta is needed + Assert.assertEquals(column.getDouble(i), expected[i], 0); + } + } + + @Test + public void testNotRemovableAggregationReComputation() { + List outputDataTypes = Arrays.asList(TSDataType.INT32, TSDataType.INT32); + int[] expected = {0, 0, 0, 1, 2, 3, 4, 5, 6, 7}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 2, 2); + AggregationWindowFunction function = + FunctionTestUtils.createAggregationWindowFunction( + TAggregationType.MIN, TSDataType.INT32, TSDataType.INT32, true); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, frameInfo); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + // This floating point are integers, no delta is needed + Assert.assertEquals(column.getInt(i), expected[i], 0); + } + } + + @Test + public void testAggregationNoReComputation() { + List outputDataTypes = Arrays.asList(TSDataType.INT32, TSDataType.DOUBLE); + double[] expected = {3, 6, 10, 15, 20, 25, 30, 35, 30, 24}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 2, 2); + AggregationWindowFunction function = + FunctionTestUtils.createAggregationWindowFunction( + TAggregationType.SUM, TSDataType.INT32, TSDataType.DOUBLE, true); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, frameInfo); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + // This floating point are integers, no delta is needed + Assert.assertEquals(column.getDouble(i), expected[i], 0); + } + } + + @Test + public void testAggregationReComputation() { + List inputDataTypes = Collections.singletonList(TSDataType.INT32); + int[] inputs = {1, 1, 1, 1, 3, 3, 3, 3, 5, 5}; + + List outputDataTypes = Arrays.asList(TSDataType.INT32, TSDataType.DOUBLE); + double[] expected = {4, 4, 4, 4, 12, 12, 12, 12, 10, 10}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + AggregationWindowFunction function = + FunctionTestUtils.createAggregationWindowFunction( + TAggregationType.SUM, TSDataType.INT32, TSDataType.DOUBLE, true); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.CURRENT_ROW, + -1, + FrameInfo.FrameBoundType.CURRENT_ROW, + -1, + 0, + SortOrder.ASC_NULLS_FIRST); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor( + tsBlock, inputDataTypes, function, frameInfo, Collections.singletonList(0)); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + // This floating point are integers, no delta is needed + Assert.assertEquals(column.getDouble(i), expected[i], 0); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/CumeDistFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/CumeDistFunctionTest.java new file mode 100644 index 000000000000..5962c44a4b3d --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/CumeDistFunctionTest.java @@ -0,0 +1,72 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class CumeDistFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + private final int[] inputs = {1, 1, 2, 2, 3}; + + private final List outputDataTypes = + Arrays.asList(TSDataType.INT32, TSDataType.DOUBLE); + private final double[] expected = {0.4, 0.4, 0.8, 0.8, 1}; + + @Test + public void testCumeDistFunction() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + CumeDistFunction function = new CumeDistFunction(); + List sortedColumns = Collections.singletonList(0); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, sortedColumns); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + // 0.4, 0.8 and 1 do not need delta + Assert.assertEquals(column.getDouble(i), expected[i], 0); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/DenseRankFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/DenseRankFunctionTest.java new file mode 100644 index 000000000000..576ea56e931c --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/DenseRankFunctionTest.java @@ -0,0 +1,71 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class DenseRankFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + private final int[] inputs = {0, 1, 1, 1, 2, 2, 3, 4, 4, 5}; + + private final List outputDataTypes = + Arrays.asList(TSDataType.INT32, TSDataType.INT64); + private final int[] expected = {1, 2, 2, 2, 3, 3, 4, 5, 5, 6}; + + @Test + public void testDenseRankFunction() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + DenseRankFunction function = new DenseRankFunction(); + List sortedColumns = Collections.singletonList(0); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, sortedColumns); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + Assert.assertEquals(column.getLong(i), expected[i]); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/NTileFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/NTileFunctionTest.java new file mode 100644 index 000000000000..92916a258a3b --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/NTileFunctionTest.java @@ -0,0 +1,128 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class NTileFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + private final List outputDataTypes = + Arrays.asList(TSDataType.INT32, TSDataType.INT64); + + @Test + public void testNTileFunctionWhenNIsLarge() { + int n = 5; + int[] inputs = {1, 2, 3}; + int[] expected = {1, 2, 3}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + NTileFunction function = new NTileFunction(n); + List sortedColumns = Collections.singletonList(0); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, sortedColumns); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + Assert.assertEquals(column.getLong(i), expected[i]); + } + } + + @Test + public void testNTileFunctionUniform() { + int n = 3; + int[] inputs = {1, 2, 3, 4, 5, 6}; + int[] expected = {1, 1, 2, 2, 3, 3}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + NTileFunction function = new NTileFunction(n); + List sortedColumns = Collections.singletonList(0); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, sortedColumns); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + Assert.assertEquals(column.getLong(i), expected[i]); + } + } + + @Test + public void testNTileFunctionNonUniform() { + int n = 3; + int[] inputs = {1, 2, 3, 4, 5, 6, 7}; + int[] expected = {1, 1, 1, 2, 2, 3, 3}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + NTileFunction function = new NTileFunction(n); + List sortedColumns = Collections.singletonList(0); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, sortedColumns); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + Assert.assertEquals(column.getLong(i), expected[i]); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/PercentRankFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/PercentRankFunctionTest.java new file mode 100644 index 000000000000..7883047e3bdc --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/PercentRankFunctionTest.java @@ -0,0 +1,72 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class PercentRankFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + private final int[] inputs = {1, 1, 2, 2, 3}; + + private final List outputDataTypes = + Arrays.asList(TSDataType.INT32, TSDataType.DOUBLE); + private final double[] expected = {0, 0, 0.5, 0.5, 1}; + + @Test + public void testPercentRankFunction() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + PercentRankFunction function = new PercentRankFunction(); + List sortedColumns = Collections.singletonList(0); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, sortedColumns); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + // 0, 0.5 and 1 do not need delta + Assert.assertEquals(column.getDouble(i), expected[i], 0); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RankFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RankFunctionTest.java new file mode 100644 index 000000000000..75c805a90bb7 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RankFunctionTest.java @@ -0,0 +1,71 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class RankFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + private final int[] inputs = {0, 1, 1, 1, 2, 2, 3, 4, 4, 5}; + + private final List outputDataTypes = + Arrays.asList(TSDataType.INT32, TSDataType.INT64); + private final int[] expected = {1, 2, 2, 2, 5, 5, 7, 8, 8, 10}; + + @Test + public void testRankFunction() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + RankFunction function = new RankFunction(); + List sortedColumns = Collections.singletonList(0); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, sortedColumns); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + Assert.assertEquals(column.getLong(i), expected[i]); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RowNumberFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RowNumberFunctionTest.java new file mode 100644 index 000000000000..ce960505bf5b --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/rank/RowNumberFunctionTest.java @@ -0,0 +1,70 @@ +/* + * 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.process.window.function.rank; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class RowNumberFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + private final int[] inputs = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}; + + private final List outputDataTypes = + Arrays.asList(TSDataType.INT32, TSDataType.INT64); + private final int[] expected = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; + + @Test + public void testRowNumberFunction() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + RowNumberFunction function = new RowNumberFunction(); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + Assert.assertEquals(column.getLong(i), expected[i]); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/FirstValueFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/FirstValueFunctionTest.java new file mode 100644 index 000000000000..52a6011e3efc --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/FirstValueFunctionTest.java @@ -0,0 +1,120 @@ +/* + * 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.process.window.function.value; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.FrameInfo; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class FirstValueFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + // Inputs element less than 0 means this pos is null + private final int[] inputs = {0, -1, -1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1, -1, -1}; + + private final List outputDataTypes = + Arrays.asList(TSDataType.INT32, TSDataType.INT32); + + @Test + public void testFirstValueFunctionIgnoreNull() { + int[] expected = {0, 0, 0, 1, 1, 1, 2, 3, 3, 4, 5, 5, 6, -1, -1, -1}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs, 2, 2); + FirstValueFunction function = new FirstValueFunction(0, true); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, frameInfo); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(column.getInt(i), expected[i]); + } + } + } + + @Test + public void testFirstValueFunctionNotIgnoreNull() { + int[] expected = {0, 0, 0, -1, -1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs, 2, 2); + FirstValueFunction function = new FirstValueFunction(0, false); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, frameInfo); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(column.getInt(i), expected[i]); + } + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LagFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LagFunctionTest.java new file mode 100644 index 000000000000..367eb15bb6c5 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LagFunctionTest.java @@ -0,0 +1,159 @@ +/* + * 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.process.window.function.value; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class LagFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + // Inputs element less than 0 means this pos is null + private final int[] inputs = {0, -1, -1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1, -1, -1}; + + private final List outputDataTypes = + Arrays.asList(TSDataType.INT32, TSDataType.INT32); + + @Test + public void testLagFunctionIgnoreNullWithoutDefault() { + int[] expected = {-1, -1, -1, -1, 0, 1, 1, 2, 3, 3, 4, 5, 5, 5, 5, 5}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs); + LagFunction function = new LagFunction(0, 2, null, true); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(expected[i], column.getInt(i)); + } + } + } + + @Test + public void testLagFunctionIgnoreNullWithDefault() { + int[] expected = {10, 10, 10, 10, 0, 1, 1, 2, 3, 3, 4, 5, 5, 5, 5, 5}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs); + LagFunction function = new LagFunction(0, 2, 10, true); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + Assert.assertEquals(expected[i], column.getInt(i)); + } + } + + @Test + public void testLagFunctionNotIgnoreNullWithoutDefault() { + int[] expected = {-1, -1, 0, -1, -1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs); + LagFunction function = new LagFunction(0, 2, null, false); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(expected[i], column.getInt(i)); + } + } + } + + @Test + public void testLagFunctionNotIgnoreNullWithDefault() { + int[] expected = {10, 10, 0, -1, -1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs); + LagFunction function = new LagFunction(0, 2, 10, false); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(expected[i], column.getInt(i)); + } + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LastValueFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LastValueFunctionTest.java new file mode 100644 index 000000000000..64f1269917b8 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LastValueFunctionTest.java @@ -0,0 +1,120 @@ +/* + * 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.process.window.function.value; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.FrameInfo; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class LastValueFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + // Inputs element less than 0 means this pos is null + private final int[] inputs = {0, -1, -1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1, -1, -1}; + + private final List outputDataTypes = + Arrays.asList(TSDataType.INT32, TSDataType.INT32); + + @Test + public void testLastValueFunctionIgnoreNull() { + int[] expected = {0, 1, 2, 2, 3, 4, 4, 5, 6, 6, 6, 6, 6, -1, -1, -1}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs, 2, 2); + LastValueFunction function = new LastValueFunction(0, true); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, frameInfo); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(column.getInt(i), expected[i]); + } + } + } + + @Test + public void testLastValueFunctionNotIgnoreNull() { + int[] expected = {-1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1, -1, -1, -1, -1}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs, 2, 2); + LastValueFunction function = new LastValueFunction(0, false); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, frameInfo); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(column.getInt(i), expected[i]); + } + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LeadFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LeadFunctionTest.java new file mode 100644 index 000000000000..90deb02b4151 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/LeadFunctionTest.java @@ -0,0 +1,159 @@ +/* + * 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.process.window.function.value; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class LeadFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + // Inputs element less than 0 means this pos is null + private final int[] inputs = {0, -1, -1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1, -1, -1}; + + private final List outputDataTypes = + Arrays.asList(TSDataType.INT32, TSDataType.INT32); + + @Test + public void testLeadFunctionIgnoreNullWithoutDefault() { + int[] expected = {2, 2, 2, 3, 4, 4, 5, 6, 6, -1, -1, -1, -1, -1, -1, -1}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs); + LeadFunction function = new LeadFunction(0, 2, null, true); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(expected[i], column.getInt(i)); + } + } + } + + @Test + public void testLeadFunctionIgnoreNullWithDefault() { + int[] expected = {2, 2, 2, 3, 4, 4, 5, 6, 6, 10, 10, 10, 10, 10, 10, 10}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs); + LeadFunction function = new LeadFunction(0, 2, 10, true); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + Assert.assertEquals(expected[i], column.getInt(i)); + } + } + + @Test + public void testLeadFunctionNotIgnoreNullWithoutDefault() { + int[] expected = {-1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1, -1, -1, -1, -1}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs); + LeadFunction function = new LeadFunction(0, 2, null, false); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(expected[i], column.getInt(i)); + } + } + } + + @Test + public void testLeadFunctionNotIgnoreNullWithDefault() { + int[] expected = {-1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1, -1, -1, 10, 10}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs); + LeadFunction function = new LeadFunction(0, 2, 10, false); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(expected[i], column.getInt(i)); + } + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/NthValueFunctionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/NthValueFunctionTest.java new file mode 100644 index 000000000000..b350a2a7aa39 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/function/value/NthValueFunctionTest.java @@ -0,0 +1,150 @@ +/* + * 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.process.window.function.value; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.function.FunctionTestUtils; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.PartitionExecutor; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.frame.FrameInfo; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.TableScanOperator.TIME_COLUMN_TEMPLATE; + +public class NthValueFunctionTest { + private final List inputDataTypes = Collections.singletonList(TSDataType.INT32); + // Inputs element less than 0 means this pos is null + private final int[] inputs = {0, -1, -1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1, -1, -1}; + + private final List outputDataTypes = + Arrays.asList(TSDataType.INT32, TSDataType.INT32); + + @Test + public void testNthValueFunctionIgnoreNull() { + int[] expected = {-1, -1, 2, -1, 3, 3, 4, 5, 5, 6, -1, -1, -1, -1, -1, -1}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs, 2, 2); + NthValueFunction function = new NthValueFunction(3, 0, true); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, frameInfo); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(expected[i], column.getInt(i)); + } + } + } + + @Test + public void testNthValueFunctionNotIgnoreNull() { + int[] expected = {-1, -1, -1, 1, 2, -1, 3, 4, -1, 5, 6, -1, -1, -1, -1, -1}; + + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs, 2, 2); + NthValueFunction function = new NthValueFunction(3, 0, false); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, frameInfo); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(expected.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), expected.length); + for (int i = 0; i < expected.length; i++) { + if (expected[i] < 0) { + Assert.assertTrue(column.isNull(i)); + } else { + Assert.assertEquals(column.getInt(i), expected[i]); + } + } + } + + @Test + public void testNthValueFunctionNotIgnoreNullOutOfBounds() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(inputs, 2, 2); + NthValueFunction function = new NthValueFunction(10, 0, false); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + PartitionExecutor partitionExecutor = + FunctionTestUtils.createPartitionExecutor(tsBlock, inputDataTypes, function, frameInfo); + + TsBlockBuilder tsBlockBuilder = new TsBlockBuilder(inputs.length, outputDataTypes); + while (partitionExecutor.hasNext()) { + partitionExecutor.processNextRow(tsBlockBuilder); + } + + TsBlock result = + tsBlockBuilder.build( + new RunLengthEncodedColumn(TIME_COLUMN_TEMPLATE, tsBlockBuilder.getPositionCount())); + Column column = result.getColumn(1); + + Assert.assertEquals(column.getPositionCount(), inputs.length); + for (int i = 0; i < inputs.length; i++) { + Assert.assertTrue(column.isNull(i)); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/FrameTestUtils.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/FrameTestUtils.java new file mode 100644 index 000000000000..d8a55d45aa29 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/FrameTestUtils.java @@ -0,0 +1,139 @@ +/* + * 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.process.window.partition.frame; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.partition.Partition; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.ColumnList; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.Range; +import org.apache.iotdb.db.queryengine.execution.operator.process.window.utils.RowComparator; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class FrameTestUtils { + private final Partition partition; + private final List sortedColumns; + + private final int partitionStart; + private final int partitionEnd; + + private int currentGroupIndex = -1; + private int peerGroupStart; + private int peerGroupEnd; + + private final RowComparator peerGroupComparator; + private final Frame frame; + + private final List frameStarts; + private final List frameEnds; + + public FrameTestUtils(TsBlock tsBlock, TSDataType inputDataType, FrameInfo frameInfo) { + this.partition = tsBlockToPartition(tsBlock); + this.sortedColumns = this.partition.getSortedColumnList(Collections.singletonList(0)); + this.partitionStart = 0; + this.partitionEnd = tsBlock.getPositionCount(); + + this.peerGroupComparator = new RowComparator(Collections.singletonList(inputDataType)); + + updatePeerGroup(0); + this.frame = createFrame(frameInfo); + + this.frameStarts = new ArrayList<>(); + this.frameEnds = new ArrayList<>(); + } + + public void processAllRows() { + for (int i = partitionStart; i < partitionEnd; i++) { + if (i == peerGroupEnd) { + updatePeerGroup(i); + } + + Range range = frame.getRange(i, currentGroupIndex, peerGroupStart, peerGroupEnd); + this.frameStarts.add(range.getStart()); + this.frameEnds.add(range.getEnd()); + } + } + + public List getFrameStarts() { + return frameStarts; + } + + public List getFrameEnds() { + return frameEnds; + } + + private void updatePeerGroup(int index) { + currentGroupIndex++; + peerGroupStart = index; + // Find end of peer group + peerGroupEnd = peerGroupStart + 1; + while (peerGroupEnd < partitionEnd + && peerGroupComparator.equalColumnLists(sortedColumns, peerGroupStart, peerGroupEnd)) { + peerGroupEnd++; + } + } + + private Partition tsBlockToPartition(TsBlock tsBlock) { + return new Partition(Collections.singletonList(tsBlock), 0, tsBlock.getPositionCount()); + } + + private List tsBlockToColumnLists(TsBlock tsBlock) { + Column[] allColumns = tsBlock.getValueColumns(); + + List columnLists = new ArrayList<>(); + for (Column column : allColumns) { + ColumnList columnList = new ColumnList(Collections.singletonList(column)); + columnLists.add(columnList); + } + + return columnLists; + } + + private Frame createFrame(FrameInfo frameInfo) { + Frame frame; + switch (frameInfo.getFrameType()) { + case RANGE: + frame = new RangeFrame(partition, frameInfo, sortedColumns, peerGroupComparator); + break; + case ROWS: + frame = new RowsFrame(partition, frameInfo, partitionStart, partitionEnd); + break; + case GROUPS: + frame = + new GroupsFrame( + partition, + frameInfo, + sortedColumns, + peerGroupComparator, + peerGroupEnd - partitionStart - 1); + break; + default: + // Unreachable + throw new UnsupportedOperationException("Unreachable!"); + } + + return frame; + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/GroupsFrameTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/GroupsFrameTest.java new file mode 100644 index 000000000000..e937f147c58d --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/GroupsFrameTest.java @@ -0,0 +1,359 @@ +/* + * 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.process.window.partition.frame; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class GroupsFrameTest { + private final int[] inputs = {1, 1, 2, 3, 3, 3}; + private final TSDataType dataType = TSDataType.INT32; + + @Test + public void testUnboundPrecedingAndPreceding() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.PRECEDING, + 1); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {-1, -1, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {-1, -1, 1, 2, 2, 2}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndCurrentRow() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.CURRENT_ROW); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 2, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.FOLLOWING, + 1); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {2, 2, 5, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndUnboundFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {5, 5, 5, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndPreceding() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 2, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.PRECEDING, + 2); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {-1, -1, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {-1, -1, 1, 2, 2, 2}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndCurrentRow() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.CURRENT_ROW); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 2, 2, 2}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 2, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 2, 2, 2}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {2, 2, 5, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndUnboundFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 2, 2, 2}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {5, 5, 5, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndCurrentRow() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.CURRENT_ROW); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 3, 3, 3}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 2, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.FOLLOWING, + 1); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 3, 3, 3}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {2, 2, 5, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndUnboundFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 3, 3, 3}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {5, 5, 5, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {2, 2, 3, -1, -1, -1}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {5, 5, 5, -1, -1, -1}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndUnboundFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.GROUPS, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING); + + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {2, 2, 3, -1, -1, -1}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {5, 5, 5, -1, -1, -1}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RangeFrameTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RangeFrameTest.java new file mode 100644 index 000000000000..4c2885fccc13 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RangeFrameTest.java @@ -0,0 +1,1127 @@ +/* + * 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.process.window.partition.frame; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; +import org.apache.iotdb.db.queryengine.plan.relational.planner.SortOrder; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class RangeFrameTest { + // No PRECEDING and FOLLOWING involved + private final int[] inputs = {1, 1, 2, 3, 3, 3, 5, 5}; + + // For PRECEDING or FOLLOWING + private final int[] ascNullsFirst = {-1, -1, 1, 4, 4, 5, 7, 7}; + private final int[] ascNullsLast = {1, 4, 4, 5, 7, 7, -1, -1}; + private final int[] descNullsFirst = {-1, -1, 7, 7, 5, 4, 4, 1}; + private final int[] descNullsLast = {7, 7, 5, 4, 4, 1, -1, -1}; + + private final TSDataType dataType = TSDataType.INT32; + + @Test + public void testUnboundPrecedingAndPrecedingAscNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsFirst, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.PRECEDING, + 1, + 0, + SortOrder.ASC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 1, 2, 2, 4, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndPrecedingAscNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsLast, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.PRECEDING, + 1, + 0, + SortOrder.ASC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {-1, 0, 0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {-1, 0, 0, 2, 3, 3, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndPrecedingDescNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsFirst, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.PRECEDING, + 1, + 0, + SortOrder.DESC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 1, 1, 3, 4, 4, 6}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndPrecedingDescNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsLast, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.PRECEDING, + 1, + 0, + SortOrder.DESC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {-1, -1, 0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {-1, -1, 1, 2, 2, 4, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndCurrentRow() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.CURRENT_ROW); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 2, 5, 5, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndFollowingAscNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsFirst, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + 0, + SortOrder.ASC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 2, 5, 5, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndFollowingAscNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsLast, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + 0, + SortOrder.ASC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {0, 3, 3, 3, 5, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndFollowingDescNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsFirst, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + 0, + SortOrder.DESC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 3, 3, 6, 6, 6, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndFollowingDescNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsLast, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + 0, + SortOrder.DESC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 4, 4, 4, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndUnboundFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {7, 7, 7, 7, 7, 7, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndPrecedingAscNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsFirst, 2, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.PRECEDING, + 2, + 0, + SortOrder.ASC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, -1, -1, -1, 3, 5, 5}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, -1, -1, -1, 4, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndPrecedingAscNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsLast, 2, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.PRECEDING, + 2, + 0, + SortOrder.ASC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {-1, -1, -1, 1, 3, 3, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {-1, -1, -1, 2, 3, 3, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndPrecedingDescNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsFirst, 2, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.PRECEDING, + 2, + 0, + SortOrder.DESC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, -1, -1, 2, 4, 4, -1}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, -1, -1, 3, 4, 4, -1}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndPrecedingDescNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsLast, 2, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.PRECEDING, + 2, + 0, + SortOrder.DESC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {-1, -1, 0, 2, 2, -1, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {-1, -1, 1, 2, 2, -1, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndCurrentRowAscNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsFirst, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.CURRENT_ROW, + 0, + SortOrder.ASC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 3, 3, 3, 5, 5}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 2, 4, 4, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndCurrentRowAscNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsLast, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.CURRENT_ROW, + 0, + SortOrder.ASC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 1, 1, 1, 3, 3, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {0, 2, 2, 3, 5, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndCurrentRowDescNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsFirst, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.CURRENT_ROW, + 0, + SortOrder.DESC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 2, 2, 4, 4, 7}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 3, 3, 4, 6, 6, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndCurrentRowDescNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsLast, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.CURRENT_ROW, + 0, + SortOrder.DESC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 2, 2, 5, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 2, 4, 4, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndFollowingAscNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsFirst, 1, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2, + 0, + SortOrder.ASC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 3, 3, 3, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 2, 5, 5, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndFollowingAscNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsLast, 1, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2, + 0, + SortOrder.ASC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 1, 1, 1, 4, 4, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {0, 3, 3, 3, 5, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndFollowingDescNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsFirst, 1, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2, + 0, + SortOrder.DESC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 2, 4, 4, 4, 7}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 3, 3, 6, 6, 6, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndFollowingDescNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsLast, 1, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2, + 0, + SortOrder.DESC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 2, 2, 5, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 4, 4, 4, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndUnboundFollowingAscNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsFirst, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING, + 0, + SortOrder.ASC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 3, 3, 3, 5, 5}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {7, 7, 7, 7, 7, 7, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndUnboundFollowingAscNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsLast, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING, + 0, + SortOrder.ASC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 1, 1, 1, 3, 3, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {7, 7, 7, 7, 7, 7, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndUnboundFollowingDescNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsFirst, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING, + 0, + SortOrder.DESC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 2, 2, 4, 4, 7}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {7, 7, 7, 7, 7, 7, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndUnboundFollowingDescNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsLast, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING, + 0, + SortOrder.DESC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 2, 2, 5, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {7, 7, 7, 7, 7, 7, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndCurrentRow() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.CURRENT_ROW); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 3, 3, 3, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 2, 5, 5, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndFollowingAscNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsFirst, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + 0, + SortOrder.ASC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 3, 3, 5, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 2, 5, 5, 7, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndFollowingAscNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsLast, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + 0, + SortOrder.ASC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 1, 1, 3, 4, 4, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {0, 3, 3, 5, 5, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndFollowingDescNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsFirst, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + 0, + SortOrder.DESC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 2, 4, 5, 5, 7}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 4, 4, 6, 6, 6, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndFollowingDescNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsLast, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + 0, + SortOrder.DESC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 3, 3, 5, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {2, 2, 4, 4, 4, 5, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndUnboundFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 2, 3, 3, 3, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {7, 7, 7, 7, 7, 7, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndFollowingAscNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsFirst, 1, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2, + 0, + SortOrder.ASC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, -1, 5, 5, 6, -1, -1}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, -1, 5, 5, 7, -1, -1}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndFollowingAscNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsLast, 1, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2, + 0, + SortOrder.ASC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {-1, 3, 3, 4, -1, -1, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {-1, 3, 3, 5, -1, -1, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndFollowingDescNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsFirst, 1, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2, + 0, + SortOrder.DESC_NULLS_FIRST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 4, 4, 5, -1, -1, -1}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 1, 4, 4, 6, -1, -1, -1}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndFollowingDescNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsLast, 1, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2, + 0, + SortOrder.DESC_NULLS_LAST); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {2, 2, 3, -1, -1, -1, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {2, 2, 4, -1, -1, -1, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndUnboundFollowingAscNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsFirst, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING, + 0, + SortOrder.ASC_NULLS_FIRST); + + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 3, 6, 6, 6, -1, -1}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {7, 7, 7, 7, 7, 7, -1, -1}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndUnboundFollowingAscNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(ascNullsLast, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING, + 0, + SortOrder.ASC_NULLS_LAST); + + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {1, 4, 4, 4, 6, 6, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {7, 7, 7, 7, 7, 7, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndUnboundFollowingDescNullsFirst() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsFirst, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING, + 0, + SortOrder.DESC_NULLS_FIRST); + + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 4, 4, 7, 7, 7, -1}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {7, 7, 7, 7, 7, 7, 7, -1}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndUnboundFollowingDescNullsLast() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithNulls(descNullsLast, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.RANGE, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING, + 0, + SortOrder.DESC_NULLS_LAST); + + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {2, 2, 5, 5, 5, 6, 6, 6}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {7, 7, 7, 7, 7, 7, 7, 7}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RowsFrameTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RowsFrameTest.java new file mode 100644 index 000000000000..19b0daeee49f --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/window/partition/frame/RowsFrameTest.java @@ -0,0 +1,359 @@ +/* + * 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.process.window.partition.frame; + +import org.apache.iotdb.db.queryengine.execution.operator.process.window.TableWindowOperatorTestUtils; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.block.TsBlock; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class RowsFrameTest { + private final int[] inputs = {0, 1, 2, 3, 4, 5}; + private final TSDataType dataType = TSDataType.INT32; + + @Test + public void testUnboundPrecedingAndPreceding() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.PRECEDING, + 1); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {-1, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {-1, 0, 1, 2, 3, 4}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndCurrentRow() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.CURRENT_ROW); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {0, 1, 2, 3, 4, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.FOLLOWING, + 1); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 2, 3, 4, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testUnboundPrecedingAndUnboundFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.UNBOUNDED_PRECEDING, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 0, 0, 0, 0}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {5, 5, 5, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndPreceding() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 2, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.PRECEDING, + 2); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {-1, 0, 0, 1, 2, 3}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {-1, 0, 1, 2, 3, 4}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndCurrentRow() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.CURRENT_ROW); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 1, 2, 3, 4}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {0, 1, 2, 3, 4, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 1, 2, 3, 4}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 2, 3, 4, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testPrecedingAndUnboundFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.PRECEDING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 0, 1, 2, 3, 4}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {5, 5, 5, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndCurrentRow() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.CURRENT_ROW); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 1, 2, 3, 4, 5}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {0, 1, 2, 3, 4, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.FOLLOWING, + 1); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 1, 2, 3, 4, 5}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {1, 2, 3, 4, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testCurrentRowAndUnboundFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.CURRENT_ROW, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {0, 1, 2, 3, 4, 5}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {5, 5, 5, 5, 5, 5}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1, 2); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.FOLLOWING, + 2); + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {1, 2, 3, 4, 5, -1}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {2, 3, 4, 5, 5, -1}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } + + @Test + public void testFollowingAndUnboundFollowing() { + TsBlock tsBlock = TableWindowOperatorTestUtils.createIntsTsBlockWithoutNulls(inputs, 1); + FrameInfo frameInfo = + new FrameInfo( + FrameInfo.FrameType.ROWS, + FrameInfo.FrameBoundType.FOLLOWING, + 1, + FrameInfo.FrameBoundType.UNBOUNDED_FOLLOWING); + + FrameTestUtils utils = new FrameTestUtils(tsBlock, dataType, frameInfo); + utils.processAllRows(); + + int[] expectedStarts = {1, 2, 3, 4, 5, -1}; + List actualStarts = utils.getFrameStarts(); + for (int i = 0; i < expectedStarts.length; i++) { + Assert.assertEquals(expectedStarts[i], (int) actualStarts.get(i)); + } + + int[] expectedEnds = {5, 5, 5, 5, 5, -1}; + List actualEnds = utils.getFrameEnds(); + for (int i = 0; i < expectedEnds.length; i++) { + Assert.assertEquals(expectedEnds[i], (int) actualEnds.get(i)); + } + } +}