/** * @return the unused section of the page, or null if fully applied. pagesIndex guaranteed to have * at least one row after this method returns */ private Page updatePagesIndex(Page page) { checkArgument(page.getPositionCount() > 0); // TODO: Fix pagesHashStrategy to allow specifying channels for comparison, it currently // requires us to rearrange the right side blocks in consecutive channel order Page preGroupedPage = rearrangePage(page, preGroupedChannels); if (pagesIndex.getPositionCount() == 0 || pagesIndex.positionEqualsRow( preGroupedPartitionHashStrategy, 0, 0, preGroupedPage.getBlocks())) { // Find the position where the pre-grouped columns change int groupEnd = findGroupEnd(preGroupedPage, preGroupedPartitionHashStrategy, 0); // Add the section of the page that contains values for the current group pagesIndex.addPage(page.getRegion(0, groupEnd)); if (page.getPositionCount() - groupEnd > 0) { // Save the remaining page, which may contain multiple partitions return page.getRegion(groupEnd, page.getPositionCount() - groupEnd); } else { // Page fully consumed return null; } } else { // We had previous results buffered, but the new page starts with new group values return page; } }
private void sortPagesIndexIfNecessary() { if (pagesIndex.getPositionCount() > 1 && !orderChannels.isEmpty()) { int startPosition = 0; while (startPosition < pagesIndex.getPositionCount()) { int endPosition = findGroupEnd(pagesIndex, preSortedPartitionHashStrategy, startPosition); pagesIndex.sort(orderChannels, ordering, startPosition, endPosition); startPosition = endPosition; } } }
private Page extractOutput() { // INVARIANT: pagesIndex contains the full grouped & sorted data for one or more partitions // Iterate through the positions sequentially until we have one full page while (!pageBuilder.isFull()) { if (partition == null || !partition.hasNext()) { int partitionStart = partition == null ? 0 : partition.getPartitionEnd(); if (partitionStart >= pagesIndex.getPositionCount()) { // Finished all of the partitions in the current pagesIndex partition = null; pagesIndex.clear(); // Try to extract more partitions from the pendingInput if (pendingInput != null && processPendingInput()) { partitionStart = 0; } else if (state == State.FINISHING) { state = State.FINISHED; // Output the remaining page if we have anything buffered if (!pageBuilder.isEmpty()) { Page page = pageBuilder.build(); pageBuilder.reset(); return page; } return null; } else { state = State.NEEDS_INPUT; return null; } } int partitionEnd = findGroupEnd(pagesIndex, unGroupedPartitionHashStrategy, partitionStart); partition = new WindowPartition( pagesIndex, partitionStart, partitionEnd, outputChannels, windowFunctions, frameInfo, peerGroupHashStrategy); } partition.processNextRow(pageBuilder); } Page page = pageBuilder.build(); pageBuilder.reset(); return page; }
@Override public Page getOutput() { if (state == State.NEEDS_INPUT || state == State.FINISHED) { return null; } Page page = extractOutput(); operatorContext.setMemoryReservation(pagesIndex.getEstimatedSize().toBytes()); return page; }
@Override public void addInput(Page page) { checkState(state == State.NEEDS_INPUT, "Operator can not take input at this time"); requireNonNull(page, "page is null"); checkState(pendingInput == null, "Operator already has pending input"); if (page.getPositionCount() == 0) { return; } pendingInput = page; if (processPendingInput()) { state = State.HAS_OUTPUT; } operatorContext.setMemoryReservation(pagesIndex.getEstimatedSize().toBytes()); }
// Assumes input grouped on relevant pagesHashStrategy columns private static int findGroupEnd( PagesIndex pagesIndex, PagesHashStrategy pagesHashStrategy, int startPosition) { checkArgument(pagesIndex.getPositionCount() > 0, "Must have at least one position"); checkPositionIndex(startPosition, pagesIndex.getPositionCount(), "startPosition out of bounds"); // Short circuit if the whole page has the same value if (pagesIndex.positionEqualsPosition( pagesHashStrategy, startPosition, pagesIndex.getPositionCount() - 1)) { return pagesIndex.getPositionCount(); } // TODO: do position binary search int endPosition = startPosition + 1; while ((endPosition < pagesIndex.getPositionCount()) && pagesIndex.positionEqualsPosition(pagesHashStrategy, endPosition - 1, endPosition)) { endPosition++; } return endPosition; }
public WindowOperator( OperatorContext operatorContext, List<Type> sourceTypes, List<Integer> outputChannels, List<WindowFunctionDefinition> windowFunctionDefinitions, List<Integer> partitionChannels, List<Integer> preGroupedChannels, List<Integer> sortChannels, List<SortOrder> sortOrder, int preSortedChannelPrefix, FrameInfo frameInfo, int expectedPositions) { requireNonNull(operatorContext, "operatorContext is null"); requireNonNull(outputChannels, "outputChannels is null"); requireNonNull(windowFunctionDefinitions, "windowFunctionDefinitions is null"); requireNonNull(partitionChannels, "partitionChannels is null"); requireNonNull(preGroupedChannels, "preGroupedChannels is null"); checkArgument( partitionChannels.containsAll(preGroupedChannels), "preGroupedChannels must be a subset of partitionChannels"); requireNonNull(sortChannels, "sortChannels is null"); requireNonNull(sortOrder, "sortOrder is null"); checkArgument( sortChannels.size() == sortOrder.size(), "Must have same number of sort channels as sort orders"); checkArgument( preSortedChannelPrefix <= sortChannels.size(), "Cannot have more pre-sorted channels than specified sorted channels"); checkArgument( preSortedChannelPrefix == 0 || ImmutableSet.copyOf(preGroupedChannels) .equals(ImmutableSet.copyOf(partitionChannels)), "preSortedChannelPrefix can only be greater than zero if all partition channels are pre-grouped"); requireNonNull(frameInfo, "frameInfo is null"); this.operatorContext = operatorContext; this.outputChannels = Ints.toArray(outputChannels); this.windowFunctions = windowFunctionDefinitions .stream() .map(WindowFunctionDefinition::createWindowFunction) .collect(toImmutableList()); this.frameInfo = frameInfo; this.types = Stream.concat( outputChannels.stream().map(sourceTypes::get), windowFunctionDefinitions.stream().map(WindowFunctionDefinition::getType)) .collect(toImmutableList()); this.pagesIndex = new PagesIndex(sourceTypes, expectedPositions); this.preGroupedChannels = Ints.toArray(preGroupedChannels); this.preGroupedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(preGroupedChannels, Optional.<Integer>empty()); List<Integer> unGroupedPartitionChannels = partitionChannels .stream() .filter(channel -> !preGroupedChannels.contains(channel)) .collect(toImmutableList()); this.unGroupedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(unGroupedPartitionChannels, Optional.empty()); List<Integer> preSortedChannels = sortChannels.stream().limit(preSortedChannelPrefix).collect(toImmutableList()); this.preSortedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(preSortedChannels, Optional.<Integer>empty()); this.peerGroupHashStrategy = pagesIndex.createPagesHashStrategy(sortChannels, Optional.empty()); this.pageBuilder = new PageBuilder(this.types); if (preSortedChannelPrefix > 0) { // This already implies that set(preGroupedChannels) == set(partitionChannels) (enforced with // checkArgument) this.orderChannels = ImmutableList.copyOf(Iterables.skip(sortChannels, preSortedChannelPrefix)); this.ordering = ImmutableList.copyOf(Iterables.skip(sortOrder, preSortedChannelPrefix)); } else { // Otherwise, we need to sort by the unGroupedPartitionChannels and all original sort channels this.orderChannels = ImmutableList.copyOf(concat(unGroupedPartitionChannels, sortChannels)); this.ordering = ImmutableList.copyOf( concat(nCopies(unGroupedPartitionChannels.size(), ASC_NULLS_LAST), sortOrder)); } }