mirror of https://github.com/apache/druid.git
Starting on Window Functions (#13458)
* Processors for Window Processing This is an initial take on how to use Processors for Window Processing. A Processor is an interface that transforms RowsAndColumns objects. RowsAndColumns objects are essentially combinations of rows and columns. The intention is that these Processors are the start of a set of operators that more closely resemble what DB engineers would be accustomed to seeing. * Wire up windowed processors with a query type that can run them end-to-end. This code can be used to actually run a query, so yay! * Wire up windowed processors with a query type that can run them end-to-end. This code can be used to actually run a query, so yay! * Some SQL tests for window functions. Added wikipedia data to the indexes available to the SQL queries and tests validating the windowing functionality as it exists now. Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
This commit is contained in:
parent
cf472162a6
commit
83261f9641
|
@ -236,8 +236,6 @@ public class TypeStrategies
|
|||
*/
|
||||
public static final class LongTypeStrategy implements TypeStrategy<Long>
|
||||
{
|
||||
private static final Comparator<Long> COMPARATOR = Longs::compare;
|
||||
|
||||
@Override
|
||||
public int estimateSizeBytes(Long value)
|
||||
{
|
||||
|
@ -276,9 +274,9 @@ public class TypeStrategies
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compare(Long o1, Long o2)
|
||||
public int compare(Object o1, Object o2)
|
||||
{
|
||||
return COMPARATOR.compare(o1, o2);
|
||||
return Longs.compare(((Number) o1).longValue(), ((Number) o2).longValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -289,8 +287,6 @@ public class TypeStrategies
|
|||
*/
|
||||
public static final class FloatTypeStrategy implements TypeStrategy<Float>
|
||||
{
|
||||
private static final Comparator<Float> COMPARATOR = Floats::compare;
|
||||
|
||||
@Override
|
||||
public int estimateSizeBytes(Float value)
|
||||
{
|
||||
|
@ -329,9 +325,9 @@ public class TypeStrategies
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compare(Float o1, Float o2)
|
||||
public int compare(Object o1, Object o2)
|
||||
{
|
||||
return COMPARATOR.compare(o1, o2);
|
||||
return Floats.compare(((Number) o1).floatValue(), ((Number) o2).floatValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -342,7 +338,6 @@ public class TypeStrategies
|
|||
*/
|
||||
public static final class DoubleTypeStrategy implements TypeStrategy<Double>
|
||||
{
|
||||
private static final Comparator<Double> COMPARATOR = Double::compare;
|
||||
|
||||
@Override
|
||||
public int estimateSizeBytes(Double value)
|
||||
|
@ -382,9 +377,9 @@ public class TypeStrategies
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compare(Double o1, Double o2)
|
||||
public int compare(Object o1, Object o2)
|
||||
{
|
||||
return COMPARATOR.compare(o1, o2);
|
||||
return Double.compare(((Number) o1).doubleValue(), ((Number) o2).doubleValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -437,7 +432,7 @@ public class TypeStrategies
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compare(String s, String s2)
|
||||
public int compare(Object s, Object s2)
|
||||
{
|
||||
// copy of lexicographical string comparator in druid processing
|
||||
// Avoid comparisons for equal references
|
||||
|
@ -447,7 +442,7 @@ public class TypeStrategies
|
|||
return 0;
|
||||
}
|
||||
|
||||
return ORDERING.compare(s, s2);
|
||||
return ORDERING.compare((String) s, (String) s2);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -521,8 +516,11 @@ public class TypeStrategies
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compare(@Nullable Object[] o1, @Nullable Object[] o2)
|
||||
public int compare(@Nullable Object o1Obj, @Nullable Object o2Obj)
|
||||
{
|
||||
Object[] o1 = (Object[]) o1Obj;
|
||||
Object[] o2 = (Object[]) o2Obj;
|
||||
|
||||
//noinspection ArrayEquality
|
||||
if (o1 == o2) {
|
||||
return 0;
|
||||
|
|
|
@ -55,8 +55,15 @@ import java.util.Comparator;
|
|||
* Implementations of this interface should be thread safe, but may not use {@link ByteBuffer} in a thread safe manner,
|
||||
* potentially modifying positions and limits, either temporarily or permanently depending on which set of methods is
|
||||
* called.
|
||||
*
|
||||
* This interface extends {@code Comparator<Object>} instead of {@code Comparator<T>} because trying to specialize the
|
||||
* type of the comparison method can run into issues for comparators of objects that can sometimes be of a different
|
||||
* java class type. For example, {@code Comparator<Long>} cannot accept Integer objects in its comparison method
|
||||
* and there is no easy way for this interface definition to allow {@code TypeStrategy<Long>} to actually be a
|
||||
* {@code Comparator<Number>}. So, we fall back to effectively erasing the generic type and having them all be
|
||||
* {@code Comparator<Object>}.
|
||||
*/
|
||||
public interface TypeStrategy<T> extends Comparator<T>
|
||||
public interface TypeStrategy<T> extends Comparator<Object>
|
||||
{
|
||||
/**
|
||||
* Estimate the size in bytes that writing this value to memory would require. This method is not required to be
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.segment.column;
|
||||
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.primitives.Longs;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
|
@ -103,7 +104,7 @@ public class TypeStrategiesTest
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compare(String o1, String o2)
|
||||
public int compare(Object o1, Object o2)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
@ -639,10 +640,13 @@ public class TypeStrategiesTest
|
|||
|
||||
public static class NullableLongPairTypeStrategy implements TypeStrategy<NullableLongPair>
|
||||
{
|
||||
|
||||
private Ordering<NullableLongPair> ordering = Comparators.naturalNullsFirst();
|
||||
|
||||
@Override
|
||||
public int compare(NullableLongPair o1, NullableLongPair o2)
|
||||
public int compare(Object o1, Object o2)
|
||||
{
|
||||
return Comparators.<NullableLongPair>naturalNullsFirst().compare(o1, o2);
|
||||
return ordering.compare((NullableLongPair) o1, (NullableLongPair) o2);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,7 +22,7 @@ package org.apache.druid.frame.read.columnar;
|
|||
import com.google.common.primitives.Ints;
|
||||
import org.apache.datasketches.memory.Memory;
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.frame.write.columnar.ComplexFrameColumnWriter;
|
||||
import org.apache.druid.frame.write.columnar.ComplexFrameMaker;
|
||||
import org.apache.druid.frame.write.columnar.FrameColumnWriters;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -53,10 +53,10 @@ public class ComplexFrameColumnReader implements FrameColumnReader
|
|||
final Memory memory = frame.region(columnNumber);
|
||||
validate(memory, frame.numRows());
|
||||
|
||||
final int typeNameLength = memory.getInt(ComplexFrameColumnWriter.TYPE_NAME_LENGTH_POSITION);
|
||||
final int typeNameLength = memory.getInt(ComplexFrameMaker.TYPE_NAME_LENGTH_POSITION);
|
||||
final byte[] typeNameBytes = new byte[typeNameLength];
|
||||
|
||||
memory.getByteArray(ComplexFrameColumnWriter.TYPE_NAME_POSITION, typeNameBytes, 0, typeNameLength);
|
||||
memory.getByteArray(ComplexFrameMaker.TYPE_NAME_POSITION, typeNameBytes, 0, typeNameLength);
|
||||
|
||||
final String typeName = StringUtils.fromUtf8(typeNameBytes);
|
||||
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
|
||||
|
@ -84,7 +84,7 @@ public class ComplexFrameColumnReader implements FrameColumnReader
|
|||
|
||||
private void validate(final Memory region, final int numRows)
|
||||
{
|
||||
if (region.getCapacity() < ComplexFrameColumnWriter.TYPE_NAME_POSITION) {
|
||||
if (region.getCapacity() < ComplexFrameMaker.TYPE_NAME_POSITION) {
|
||||
throw new ISE("Column is not big enough for a header");
|
||||
}
|
||||
|
||||
|
@ -93,9 +93,9 @@ public class ComplexFrameColumnReader implements FrameColumnReader
|
|||
throw new ISE("Column does not have the correct type code");
|
||||
}
|
||||
|
||||
final int typeNameLength = region.getInt(ComplexFrameColumnWriter.TYPE_NAME_LENGTH_POSITION);
|
||||
final int typeNameLength = region.getInt(ComplexFrameMaker.TYPE_NAME_LENGTH_POSITION);
|
||||
if (region.getCapacity() <
|
||||
ComplexFrameColumnWriter.TYPE_NAME_POSITION + typeNameLength + (long) numRows * Integer.BYTES) {
|
||||
ComplexFrameMaker.TYPE_NAME_POSITION + typeNameLength + (long) numRows * Integer.BYTES) {
|
||||
throw new ISE("Column is missing offset section");
|
||||
}
|
||||
}
|
||||
|
@ -198,7 +198,7 @@ public class ComplexFrameColumnReader implements FrameColumnReader
|
|||
startOfDataSection + memory.getInt(startOfOffsetSection + (long) Integer.BYTES * (physicalRow - 1));
|
||||
}
|
||||
|
||||
if (memory.getByte(startOffset) == ComplexFrameColumnWriter.NULL_MARKER) {
|
||||
if (memory.getByte(startOffset) == ComplexFrameMaker.NULL_MARKER) {
|
||||
return null;
|
||||
} else {
|
||||
final int payloadLength = Ints.checkedCast(endOffset - startOffset - Byte.BYTES);
|
||||
|
|
|
@ -22,7 +22,7 @@ package org.apache.druid.frame.read.columnar;
|
|||
import org.apache.datasketches.memory.Memory;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.frame.write.columnar.DoubleFrameColumnWriter;
|
||||
import org.apache.druid.frame.write.columnar.DoubleFrameMaker;
|
||||
import org.apache.druid.frame.write.columnar.FrameColumnWriters;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
@ -69,7 +69,7 @@ public class DoubleFrameColumnReader implements FrameColumnReader
|
|||
final long memorySize = region.getCapacity();
|
||||
|
||||
// Check if column is big enough for a header
|
||||
if (memorySize < DoubleFrameColumnWriter.DATA_OFFSET) {
|
||||
if (memorySize < DoubleFrameMaker.DATA_OFFSET) {
|
||||
throw new ISE("Column is not big enough for a header");
|
||||
}
|
||||
|
||||
|
@ -79,10 +79,10 @@ public class DoubleFrameColumnReader implements FrameColumnReader
|
|||
}
|
||||
|
||||
final boolean hasNulls = getHasNulls(region);
|
||||
final int sz = DoubleFrameColumnWriter.valueSize(hasNulls);
|
||||
final int sz = DoubleFrameMaker.valueSize(hasNulls);
|
||||
|
||||
// Check column length again, now that we know exactly how long it should be.
|
||||
if (memorySize != DoubleFrameColumnWriter.DATA_OFFSET + (long) sz * numRows) {
|
||||
if (memorySize != DoubleFrameMaker.DATA_OFFSET + (long) sz * numRows) {
|
||||
throw new ISE("Column does not have the correct length");
|
||||
}
|
||||
}
|
||||
|
@ -108,9 +108,9 @@ public class DoubleFrameColumnReader implements FrameColumnReader
|
|||
{
|
||||
this.frame = frame;
|
||||
this.hasNulls = hasNulls;
|
||||
this.sz = DoubleFrameColumnWriter.valueSize(hasNulls);
|
||||
this.sz = DoubleFrameMaker.valueSize(hasNulls);
|
||||
this.memory = memory;
|
||||
this.memoryPosition = DoubleFrameColumnWriter.DATA_OFFSET;
|
||||
this.memoryPosition = DoubleFrameMaker.DATA_OFFSET;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,7 +22,7 @@ package org.apache.druid.frame.read.columnar;
|
|||
import org.apache.datasketches.memory.Memory;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.frame.write.columnar.FloatFrameColumnWriter;
|
||||
import org.apache.druid.frame.write.columnar.FloatFrameMaker;
|
||||
import org.apache.druid.frame.write.columnar.FrameColumnWriters;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
@ -69,7 +69,7 @@ public class FloatFrameColumnReader implements FrameColumnReader
|
|||
final long memorySize = region.getCapacity();
|
||||
|
||||
// Check if column is big enough for a header
|
||||
if (memorySize < FloatFrameColumnWriter.DATA_OFFSET) {
|
||||
if (memorySize < FloatFrameMaker.DATA_OFFSET) {
|
||||
throw new ISE("Column is not big enough for a header");
|
||||
}
|
||||
|
||||
|
@ -79,10 +79,10 @@ public class FloatFrameColumnReader implements FrameColumnReader
|
|||
}
|
||||
|
||||
final boolean hasNulls = getHasNulls(region);
|
||||
final int sz = FloatFrameColumnWriter.valueSize(hasNulls);
|
||||
final int sz = FloatFrameMaker.valueSize(hasNulls);
|
||||
|
||||
// Check column length again, now that we know exactly how long it should be.
|
||||
if (memorySize != FloatFrameColumnWriter.DATA_OFFSET + (long) sz * numRows) {
|
||||
if (memorySize != FloatFrameMaker.DATA_OFFSET + (long) sz * numRows) {
|
||||
throw new ISE("Column does not have the correct length");
|
||||
}
|
||||
}
|
||||
|
@ -108,9 +108,9 @@ public class FloatFrameColumnReader implements FrameColumnReader
|
|||
{
|
||||
this.frame = frame;
|
||||
this.hasNulls = hasNulls;
|
||||
this.sz = FloatFrameColumnWriter.valueSize(hasNulls);
|
||||
this.sz = FloatFrameMaker.valueSize(hasNulls);
|
||||
this.memory = memory;
|
||||
this.memoryPosition = FloatFrameColumnWriter.DATA_OFFSET;
|
||||
this.memoryPosition = FloatFrameMaker.DATA_OFFSET;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,7 +23,7 @@ import org.apache.datasketches.memory.Memory;
|
|||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.frame.write.columnar.FrameColumnWriters;
|
||||
import org.apache.druid.frame.write.columnar.LongFrameColumnWriter;
|
||||
import org.apache.druid.frame.write.columnar.LongFrameMaker;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
@ -67,7 +67,7 @@ public class LongFrameColumnReader implements FrameColumnReader
|
|||
private void validate(final Memory region, final int numRows)
|
||||
{
|
||||
// Check if column is big enough for a header
|
||||
if (region.getCapacity() < LongFrameColumnWriter.DATA_OFFSET) {
|
||||
if (region.getCapacity() < LongFrameMaker.DATA_OFFSET) {
|
||||
throw new ISE("Column is not big enough for a header");
|
||||
}
|
||||
|
||||
|
@ -77,10 +77,10 @@ public class LongFrameColumnReader implements FrameColumnReader
|
|||
}
|
||||
|
||||
final boolean hasNulls = getHasNulls(region);
|
||||
final int sz = LongFrameColumnWriter.valueSize(hasNulls);
|
||||
final int sz = LongFrameMaker.valueSize(hasNulls);
|
||||
|
||||
// Check column length again, now that we know exactly how long it should be.
|
||||
if (region.getCapacity() != LongFrameColumnWriter.DATA_OFFSET + (long) sz * numRows) {
|
||||
if (region.getCapacity() != LongFrameMaker.DATA_OFFSET + (long) sz * numRows) {
|
||||
throw new ISE("Column does not have the correct length");
|
||||
}
|
||||
}
|
||||
|
@ -106,9 +106,9 @@ public class LongFrameColumnReader implements FrameColumnReader
|
|||
{
|
||||
this.frame = frame;
|
||||
this.hasNulls = hasNulls;
|
||||
this.sz = LongFrameColumnWriter.valueSize(hasNulls);
|
||||
this.sz = LongFrameMaker.valueSize(hasNulls);
|
||||
this.memory = memory;
|
||||
this.memoryPosition = LongFrameColumnWriter.DATA_OFFSET;
|
||||
this.memoryPosition = LongFrameMaker.DATA_OFFSET;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,14 +19,9 @@
|
|||
|
||||
package org.apache.druid.frame.write.columnar;
|
||||
|
||||
import com.google.common.primitives.Ints;
|
||||
import it.unimi.dsi.fastutil.bytes.ByteArrays;
|
||||
import org.apache.datasketches.memory.WritableMemory;
|
||||
import org.apache.druid.frame.allocation.AppendableMemory;
|
||||
import org.apache.druid.frame.allocation.MemoryAllocator;
|
||||
import org.apache.druid.frame.allocation.MemoryRange;
|
||||
import org.apache.druid.frame.read.columnar.ComplexFrameColumnReader;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
||||
import org.apache.druid.segment.serde.ComplexMetricSerde;
|
||||
|
@ -38,22 +33,9 @@ import org.apache.druid.segment.serde.ComplexMetricSerde;
|
|||
*/
|
||||
public class ComplexFrameColumnWriter implements FrameColumnWriter
|
||||
{
|
||||
// Less than half of AppendableMemory.DEFAULT_INITIAL_ALLOCATION_SIZE.
|
||||
// This guarantees we can fit a WorkerMemoryParmeters.MAX_FRAME_COLUMNS number of columns into a frame.
|
||||
private static final int INITIAL_ALLOCATION_SIZE = 128;
|
||||
|
||||
public static final byte NOT_NULL_MARKER = 0x00;
|
||||
public static final byte NULL_MARKER = 0x01;
|
||||
public static final int TYPE_NAME_LENGTH_POSITION = Byte.BYTES;
|
||||
public static final int TYPE_NAME_POSITION = Byte.BYTES + Integer.BYTES;
|
||||
|
||||
private final ComplexMetricSerde serde;
|
||||
private final BaseObjectColumnValueSelector<?> selector;
|
||||
private final AppendableMemory offsetMemory;
|
||||
private final AppendableMemory dataMemory;
|
||||
private final byte[] typeNameBytes;
|
||||
|
||||
private int lastDataLength = -1;
|
||||
private final ComplexFrameMaker maker;
|
||||
|
||||
ComplexFrameColumnWriter(
|
||||
final BaseObjectColumnValueSelector<?> selector,
|
||||
|
@ -63,95 +45,37 @@ public class ComplexFrameColumnWriter implements FrameColumnWriter
|
|||
{
|
||||
this.selector = selector;
|
||||
this.serde = serde;
|
||||
this.offsetMemory = AppendableMemory.create(allocator, INITIAL_ALLOCATION_SIZE);
|
||||
this.dataMemory = AppendableMemory.create(allocator, INITIAL_ALLOCATION_SIZE);
|
||||
this.typeNameBytes = StringUtils.toUtf8(serde.getTypeName());
|
||||
this.maker = new ComplexFrameMaker(allocator, StringUtils.toUtf8(serde.getTypeName()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean addSelection()
|
||||
{
|
||||
if (!offsetMemory.reserveAdditional(Integer.BYTES)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final Object complexObject = selector.getObject();
|
||||
final byte[] complexBytes = complexObject == null ? ByteArrays.EMPTY_ARRAY : serde.toBytes(complexObject);
|
||||
|
||||
if (complexBytes.length == Integer.MAX_VALUE) {
|
||||
// Cannot handle objects this large.
|
||||
return false;
|
||||
}
|
||||
|
||||
final int dataLength = complexBytes.length + 1;
|
||||
|
||||
if (dataMemory.size() + dataLength > Integer.MAX_VALUE || !(dataMemory.reserveAdditional(dataLength))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
// All space is reserved. Start writing.
|
||||
final MemoryRange<WritableMemory> offsetCursor = offsetMemory.cursor();
|
||||
offsetCursor.memory().putInt(offsetCursor.start(), Ints.checkedCast(dataMemory.size() + dataLength));
|
||||
offsetMemory.advanceCursor(Integer.BYTES);
|
||||
|
||||
final MemoryRange<WritableMemory> dataCursor = dataMemory.cursor();
|
||||
dataCursor.memory().putByte(dataCursor.start(), complexObject == null ? NULL_MARKER : NOT_NULL_MARKER);
|
||||
dataCursor.memory().putByteArray(dataCursor.start() + 1, complexBytes, 0, complexBytes.length);
|
||||
dataMemory.advanceCursor(dataLength);
|
||||
|
||||
lastDataLength = dataLength;
|
||||
return true;
|
||||
return maker.add(complexObject == null ? null : serde.toBytes(complexObject));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void undo()
|
||||
{
|
||||
if (lastDataLength == -1) {
|
||||
throw new ISE("Nothing to undo");
|
||||
}
|
||||
|
||||
offsetMemory.rewindCursor(Integer.BYTES);
|
||||
dataMemory.rewindCursor(lastDataLength);
|
||||
lastDataLength = -1;
|
||||
maker.undo();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size()
|
||||
{
|
||||
return headerSize() + offsetMemory.size() + dataMemory.size();
|
||||
return maker.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long writeTo(final WritableMemory memory, final long startPosition)
|
||||
{
|
||||
long currentPosition = startPosition;
|
||||
|
||||
memory.putByte(currentPosition, FrameColumnWriters.TYPE_COMPLEX);
|
||||
currentPosition += 1;
|
||||
|
||||
memory.putInt(currentPosition, typeNameBytes.length);
|
||||
currentPosition += Integer.BYTES;
|
||||
|
||||
memory.putByteArray(currentPosition, typeNameBytes, 0, typeNameBytes.length);
|
||||
currentPosition += typeNameBytes.length;
|
||||
|
||||
currentPosition += offsetMemory.writeTo(memory, currentPosition);
|
||||
currentPosition += dataMemory.writeTo(memory, currentPosition);
|
||||
|
||||
return currentPosition - startPosition;
|
||||
return maker.writeTo(memory, startPosition);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
offsetMemory.close();
|
||||
dataMemory.close();
|
||||
}
|
||||
|
||||
private int headerSize()
|
||||
{
|
||||
return 1 /* type code */
|
||||
+ Integer.BYTES /* type name length */
|
||||
+ typeNameBytes.length;
|
||||
maker.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,147 @@
|
|||
/*
|
||||
* 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.druid.frame.write.columnar;
|
||||
|
||||
import com.google.common.primitives.Ints;
|
||||
import it.unimi.dsi.fastutil.bytes.ByteArrays;
|
||||
import org.apache.datasketches.memory.WritableMemory;
|
||||
import org.apache.druid.frame.allocation.AppendableMemory;
|
||||
import org.apache.druid.frame.allocation.MemoryAllocator;
|
||||
import org.apache.druid.frame.allocation.MemoryRange;
|
||||
import org.apache.druid.frame.read.columnar.ComplexFrameColumnReader;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
||||
/**
|
||||
* Column writer for complex columns.
|
||||
*
|
||||
* Dual to {@link ComplexFrameColumnReader}.
|
||||
*/
|
||||
public class ComplexFrameMaker
|
||||
{
|
||||
// Less than half of AppendableMemory.DEFAULT_INITIAL_ALLOCATION_SIZE.
|
||||
// This guarantees we can fit a WorkerMemoryParmeters.MAX_FRAME_COLUMNS number of columns into a frame.
|
||||
private static final int INITIAL_ALLOCATION_SIZE = 128;
|
||||
|
||||
public static final byte NOT_NULL_MARKER = 0x00;
|
||||
public static final byte NULL_MARKER = 0x01;
|
||||
public static final int TYPE_NAME_LENGTH_POSITION = Byte.BYTES;
|
||||
public static final int TYPE_NAME_POSITION = Byte.BYTES + Integer.BYTES;
|
||||
|
||||
private final AppendableMemory offsetMemory;
|
||||
private final AppendableMemory dataMemory;
|
||||
private final byte[] typeName;
|
||||
|
||||
private int lastDataLength = -1;
|
||||
|
||||
ComplexFrameMaker(
|
||||
final MemoryAllocator allocator,
|
||||
final byte[] typeName
|
||||
)
|
||||
{
|
||||
this.offsetMemory = AppendableMemory.create(allocator, INITIAL_ALLOCATION_SIZE);
|
||||
this.dataMemory = AppendableMemory.create(allocator, INITIAL_ALLOCATION_SIZE);
|
||||
this.typeName = typeName;
|
||||
}
|
||||
|
||||
public boolean add(byte[] bytes)
|
||||
{
|
||||
if (!offsetMemory.reserveAdditional(Integer.BYTES)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
boolean isNull = false;
|
||||
if (bytes == null) {
|
||||
isNull = true;
|
||||
bytes = ByteArrays.EMPTY_ARRAY;
|
||||
}
|
||||
|
||||
if (bytes.length == Integer.MAX_VALUE) {
|
||||
// Cannot handle objects this large.
|
||||
return false;
|
||||
}
|
||||
|
||||
final int dataLength = bytes.length + 1;
|
||||
|
||||
if (dataMemory.size() + dataLength > Integer.MAX_VALUE || !(dataMemory.reserveAdditional(dataLength))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
// All space is reserved. Start writing.
|
||||
final MemoryRange<WritableMemory> offsetCursor = offsetMemory.cursor();
|
||||
offsetCursor.memory().putInt(offsetCursor.start(), Ints.checkedCast(dataMemory.size() + dataLength));
|
||||
offsetMemory.advanceCursor(Integer.BYTES);
|
||||
|
||||
final MemoryRange<WritableMemory> dataCursor = dataMemory.cursor();
|
||||
dataCursor.memory().putByte(dataCursor.start(), isNull ? NULL_MARKER : NOT_NULL_MARKER);
|
||||
dataCursor.memory().putByteArray(dataCursor.start() + 1, bytes, 0, bytes.length);
|
||||
dataMemory.advanceCursor(dataLength);
|
||||
|
||||
lastDataLength = dataLength;
|
||||
return true;
|
||||
}
|
||||
|
||||
public void undo()
|
||||
{
|
||||
if (lastDataLength == -1) {
|
||||
throw new ISE("Nothing to undo");
|
||||
}
|
||||
|
||||
offsetMemory.rewindCursor(Integer.BYTES);
|
||||
dataMemory.rewindCursor(lastDataLength);
|
||||
lastDataLength = -1;
|
||||
}
|
||||
|
||||
public long size()
|
||||
{
|
||||
return headerSize() + offsetMemory.size() + dataMemory.size();
|
||||
}
|
||||
|
||||
public long writeTo(final WritableMemory memory, final long startPosition)
|
||||
{
|
||||
long currentPosition = startPosition;
|
||||
|
||||
memory.putByte(currentPosition, FrameColumnWriters.TYPE_COMPLEX);
|
||||
currentPosition += 1;
|
||||
|
||||
memory.putInt(currentPosition, typeName.length);
|
||||
currentPosition += Integer.BYTES;
|
||||
|
||||
memory.putByteArray(currentPosition, typeName, 0, typeName.length);
|
||||
currentPosition += typeName.length;
|
||||
|
||||
currentPosition += offsetMemory.writeTo(memory, currentPosition);
|
||||
currentPosition += dataMemory.writeTo(memory, currentPosition);
|
||||
|
||||
return currentPosition - startPosition;
|
||||
}
|
||||
|
||||
public void close()
|
||||
{
|
||||
offsetMemory.close();
|
||||
dataMemory.close();
|
||||
}
|
||||
|
||||
private int headerSize()
|
||||
{
|
||||
return 1 /* type code */
|
||||
+ Integer.BYTES /* type name length */
|
||||
+ typeName.length;
|
||||
}
|
||||
}
|
|
@ -20,19 +20,13 @@
|
|||
package org.apache.druid.frame.write.columnar;
|
||||
|
||||
import org.apache.datasketches.memory.WritableMemory;
|
||||
import org.apache.druid.frame.allocation.AppendableMemory;
|
||||
import org.apache.druid.frame.allocation.MemoryAllocator;
|
||||
import org.apache.druid.frame.allocation.MemoryRange;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
|
||||
public class DoubleFrameColumnWriter implements FrameColumnWriter
|
||||
{
|
||||
public static final long DATA_OFFSET = 1 /* type code */ + 1 /* has nulls? */;
|
||||
|
||||
private final BaseDoubleColumnValueSelector selector;
|
||||
private final AppendableMemory appendableMemory;
|
||||
private final boolean hasNulls;
|
||||
private final int sz;
|
||||
private final DoubleFrameMaker maker;
|
||||
|
||||
DoubleFrameColumnWriter(
|
||||
BaseDoubleColumnValueSelector selector,
|
||||
|
@ -41,71 +35,40 @@ public class DoubleFrameColumnWriter implements FrameColumnWriter
|
|||
)
|
||||
{
|
||||
this.selector = selector;
|
||||
this.appendableMemory = AppendableMemory.create(allocator);
|
||||
this.hasNulls = hasNulls;
|
||||
this.sz = valueSize(hasNulls);
|
||||
}
|
||||
|
||||
public static int valueSize(final boolean hasNulls)
|
||||
{
|
||||
return hasNulls ? Double.BYTES + 1 : Double.BYTES;
|
||||
this.maker = new DoubleFrameMaker(allocator, hasNulls);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean addSelection()
|
||||
{
|
||||
if (!(appendableMemory.reserveAdditional(sz))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final MemoryRange<WritableMemory> cursor = appendableMemory.cursor();
|
||||
final WritableMemory memory = cursor.memory();
|
||||
final long position = cursor.start();
|
||||
|
||||
if (hasNulls) {
|
||||
if (selector.isNull()) {
|
||||
memory.putByte(position, (byte) 1);
|
||||
memory.putDouble(position + 1, 0);
|
||||
} else {
|
||||
memory.putByte(position, (byte) 0);
|
||||
memory.putDouble(position + 1, selector.getDouble());
|
||||
}
|
||||
if (selector.isNull()) {
|
||||
return maker.addNull();
|
||||
} else {
|
||||
memory.putDouble(position, selector.getDouble());
|
||||
return maker.add(selector.getDouble());
|
||||
}
|
||||
|
||||
appendableMemory.advanceCursor(sz);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void undo()
|
||||
{
|
||||
appendableMemory.rewindCursor(sz);
|
||||
maker.undo();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size()
|
||||
{
|
||||
return DATA_OFFSET + appendableMemory.size();
|
||||
return maker.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long writeTo(final WritableMemory memory, final long startPosition)
|
||||
{
|
||||
long currentPosition = startPosition;
|
||||
|
||||
memory.putByte(currentPosition, FrameColumnWriters.TYPE_DOUBLE);
|
||||
memory.putByte(currentPosition + 1, hasNulls ? (byte) 1 : (byte) 0);
|
||||
currentPosition += 2;
|
||||
|
||||
currentPosition += appendableMemory.writeTo(memory, currentPosition);
|
||||
return currentPosition - startPosition;
|
||||
return maker.writeTo(memory, startPosition);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
appendableMemory.close();
|
||||
maker.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,122 @@
|
|||
/*
|
||||
* 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.druid.frame.write.columnar;
|
||||
|
||||
import org.apache.datasketches.memory.WritableMemory;
|
||||
import org.apache.druid.frame.allocation.AppendableMemory;
|
||||
import org.apache.druid.frame.allocation.MemoryAllocator;
|
||||
import org.apache.druid.frame.allocation.MemoryRange;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
public class DoubleFrameMaker implements Closeable
|
||||
{
|
||||
public static final long DATA_OFFSET = 1 /* type code */ + 1 /* has nulls? */;
|
||||
|
||||
private final AppendableMemory appendableMemory;
|
||||
private final boolean hasNulls;
|
||||
private final int sz;
|
||||
|
||||
public DoubleFrameMaker(
|
||||
MemoryAllocator allocator,
|
||||
boolean hasNulls
|
||||
)
|
||||
{
|
||||
this.appendableMemory = AppendableMemory.create(allocator);
|
||||
this.hasNulls = hasNulls;
|
||||
this.sz = valueSize(hasNulls);
|
||||
}
|
||||
|
||||
public static int valueSize(final boolean hasNulls)
|
||||
{
|
||||
return hasNulls ? Double.BYTES + 1 : Double.BYTES;
|
||||
}
|
||||
|
||||
public boolean add(double value)
|
||||
{
|
||||
if (!(appendableMemory.reserveAdditional(sz))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final MemoryRange<WritableMemory> cursor = appendableMemory.cursor();
|
||||
final WritableMemory memory = cursor.memory();
|
||||
final long position = cursor.start();
|
||||
|
||||
if (hasNulls) {
|
||||
memory.putByte(position, (byte) 0);
|
||||
memory.putDouble(position + 1, value);
|
||||
} else {
|
||||
memory.putDouble(position, value);
|
||||
}
|
||||
|
||||
appendableMemory.advanceCursor(sz);
|
||||
return true;
|
||||
}
|
||||
|
||||
public boolean addNull()
|
||||
{
|
||||
if (!hasNulls) {
|
||||
throw new ISE("Was told that null doesn't exist, cannot add null");
|
||||
}
|
||||
|
||||
if (!(appendableMemory.reserveAdditional(sz))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final MemoryRange<WritableMemory> cursor = appendableMemory.cursor();
|
||||
final WritableMemory memory = cursor.memory();
|
||||
final long position = cursor.start();
|
||||
|
||||
memory.putByte(position, (byte) 1);
|
||||
memory.putDouble(position + 1, 0);
|
||||
|
||||
appendableMemory.advanceCursor(sz);
|
||||
return true;
|
||||
}
|
||||
|
||||
public void undo()
|
||||
{
|
||||
appendableMemory.rewindCursor(sz);
|
||||
}
|
||||
|
||||
public long size()
|
||||
{
|
||||
return DATA_OFFSET + appendableMemory.size();
|
||||
}
|
||||
|
||||
public long writeTo(final WritableMemory memory, final long startPosition)
|
||||
{
|
||||
long currentPosition = startPosition;
|
||||
|
||||
memory.putByte(currentPosition, FrameColumnWriters.TYPE_DOUBLE);
|
||||
memory.putByte(currentPosition + 1, hasNulls ? (byte) 1 : (byte) 0);
|
||||
currentPosition += 2;
|
||||
|
||||
currentPosition += appendableMemory.writeTo(memory, currentPosition);
|
||||
return currentPosition - startPosition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
appendableMemory.close();
|
||||
}
|
||||
}
|
|
@ -20,19 +20,13 @@
|
|||
package org.apache.druid.frame.write.columnar;
|
||||
|
||||
import org.apache.datasketches.memory.WritableMemory;
|
||||
import org.apache.druid.frame.allocation.AppendableMemory;
|
||||
import org.apache.druid.frame.allocation.MemoryAllocator;
|
||||
import org.apache.druid.frame.allocation.MemoryRange;
|
||||
import org.apache.druid.segment.BaseFloatColumnValueSelector;
|
||||
|
||||
public class FloatFrameColumnWriter implements FrameColumnWriter
|
||||
{
|
||||
public static final long DATA_OFFSET = 1 /* type code */ + 1 /* has nulls? */;
|
||||
|
||||
private final BaseFloatColumnValueSelector selector;
|
||||
private final AppendableMemory appendableMemory;
|
||||
private final boolean hasNulls;
|
||||
private final int sz;
|
||||
private final FloatFrameMaker maker;
|
||||
|
||||
FloatFrameColumnWriter(
|
||||
BaseFloatColumnValueSelector selector,
|
||||
|
@ -41,71 +35,40 @@ public class FloatFrameColumnWriter implements FrameColumnWriter
|
|||
)
|
||||
{
|
||||
this.selector = selector;
|
||||
this.appendableMemory = AppendableMemory.create(allocator);
|
||||
this.hasNulls = hasNulls;
|
||||
this.sz = valueSize(hasNulls);
|
||||
}
|
||||
|
||||
public static int valueSize(final boolean hasNulls)
|
||||
{
|
||||
return hasNulls ? Float.BYTES + 1 : Float.BYTES;
|
||||
this.maker = new FloatFrameMaker(allocator, hasNulls);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean addSelection()
|
||||
{
|
||||
if (!(appendableMemory.reserveAdditional(sz))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final MemoryRange<WritableMemory> cursor = appendableMemory.cursor();
|
||||
final WritableMemory memory = cursor.memory();
|
||||
final long position = cursor.start();
|
||||
|
||||
if (hasNulls) {
|
||||
if (selector.isNull()) {
|
||||
memory.putByte(position, (byte) 1);
|
||||
memory.putFloat(position + 1, 0);
|
||||
} else {
|
||||
memory.putByte(position, (byte) 0);
|
||||
memory.putFloat(position + 1, selector.getFloat());
|
||||
}
|
||||
if (selector.isNull()) {
|
||||
return maker.addNull();
|
||||
} else {
|
||||
memory.putFloat(position, selector.getFloat());
|
||||
return maker.add(selector.getFloat());
|
||||
}
|
||||
|
||||
appendableMemory.advanceCursor(sz);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void undo()
|
||||
{
|
||||
appendableMemory.rewindCursor(sz);
|
||||
maker.undo();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size()
|
||||
{
|
||||
return DATA_OFFSET + appendableMemory.size();
|
||||
return maker.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long writeTo(final WritableMemory memory, final long startPosition)
|
||||
{
|
||||
long currentPosition = startPosition;
|
||||
|
||||
memory.putByte(currentPosition, FrameColumnWriters.TYPE_FLOAT);
|
||||
memory.putByte(currentPosition + 1, hasNulls ? (byte) 1 : (byte) 0);
|
||||
currentPosition += 2;
|
||||
|
||||
currentPosition += appendableMemory.writeTo(memory, currentPosition);
|
||||
return currentPosition - startPosition;
|
||||
return maker.writeTo(memory, startPosition);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
appendableMemory.close();
|
||||
maker.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,122 @@
|
|||
/*
|
||||
* 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.druid.frame.write.columnar;
|
||||
|
||||
import org.apache.datasketches.memory.WritableMemory;
|
||||
import org.apache.druid.frame.allocation.AppendableMemory;
|
||||
import org.apache.druid.frame.allocation.MemoryAllocator;
|
||||
import org.apache.druid.frame.allocation.MemoryRange;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
public class FloatFrameMaker implements Closeable
|
||||
{
|
||||
public static final long DATA_OFFSET = 1 /* type code */ + 1 /* has nulls? */;
|
||||
|
||||
private final AppendableMemory appendableMemory;
|
||||
private final boolean hasNulls;
|
||||
private final int sz;
|
||||
|
||||
public FloatFrameMaker(
|
||||
MemoryAllocator allocator,
|
||||
boolean hasNulls
|
||||
)
|
||||
{
|
||||
this.appendableMemory = AppendableMemory.create(allocator);
|
||||
this.hasNulls = hasNulls;
|
||||
this.sz = valueSize(hasNulls);
|
||||
}
|
||||
|
||||
public static int valueSize(final boolean hasNulls)
|
||||
{
|
||||
return hasNulls ? Float.BYTES + 1 : Float.BYTES;
|
||||
}
|
||||
|
||||
public boolean add(float value)
|
||||
{
|
||||
if (!(appendableMemory.reserveAdditional(sz))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final MemoryRange<WritableMemory> cursor = appendableMemory.cursor();
|
||||
final WritableMemory memory = cursor.memory();
|
||||
final long position = cursor.start();
|
||||
|
||||
if (hasNulls) {
|
||||
memory.putByte(position, (byte) 0);
|
||||
memory.putFloat(position + 1, value);
|
||||
} else {
|
||||
memory.putFloat(position, value);
|
||||
}
|
||||
|
||||
appendableMemory.advanceCursor(sz);
|
||||
return true;
|
||||
}
|
||||
|
||||
public boolean addNull()
|
||||
{
|
||||
if (!hasNulls) {
|
||||
throw new ISE("Was told that null doesn't exist, cannot add null");
|
||||
}
|
||||
|
||||
if (!(appendableMemory.reserveAdditional(sz))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final MemoryRange<WritableMemory> cursor = appendableMemory.cursor();
|
||||
final WritableMemory memory = cursor.memory();
|
||||
final long position = cursor.start();
|
||||
|
||||
memory.putByte(position, (byte) 1);
|
||||
memory.putFloat(position + 1, 0);
|
||||
|
||||
appendableMemory.advanceCursor(sz);
|
||||
return true;
|
||||
}
|
||||
|
||||
public void undo()
|
||||
{
|
||||
appendableMemory.rewindCursor(sz);
|
||||
}
|
||||
|
||||
public long size()
|
||||
{
|
||||
return DATA_OFFSET + appendableMemory.size();
|
||||
}
|
||||
|
||||
public long writeTo(final WritableMemory memory, final long startPosition)
|
||||
{
|
||||
long currentPosition = startPosition;
|
||||
|
||||
memory.putByte(currentPosition, FrameColumnWriters.TYPE_FLOAT);
|
||||
memory.putByte(currentPosition + 1, hasNulls ? (byte) 1 : (byte) 0);
|
||||
currentPosition += 2;
|
||||
|
||||
currentPosition += appendableMemory.writeTo(memory, currentPosition);
|
||||
return currentPosition - startPosition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
appendableMemory.close();
|
||||
}
|
||||
}
|
|
@ -20,19 +20,13 @@
|
|||
package org.apache.druid.frame.write.columnar;
|
||||
|
||||
import org.apache.datasketches.memory.WritableMemory;
|
||||
import org.apache.druid.frame.allocation.AppendableMemory;
|
||||
import org.apache.druid.frame.allocation.MemoryAllocator;
|
||||
import org.apache.druid.frame.allocation.MemoryRange;
|
||||
import org.apache.druid.segment.BaseLongColumnValueSelector;
|
||||
|
||||
public class LongFrameColumnWriter implements FrameColumnWriter
|
||||
{
|
||||
public static final long DATA_OFFSET = 1 /* type code */ + 1 /* has nulls? */;
|
||||
|
||||
private final BaseLongColumnValueSelector selector;
|
||||
private final AppendableMemory appendableMemory;
|
||||
private final boolean hasNulls;
|
||||
private final int sz;
|
||||
private final LongFrameMaker maker;
|
||||
|
||||
LongFrameColumnWriter(
|
||||
BaseLongColumnValueSelector selector,
|
||||
|
@ -41,71 +35,40 @@ public class LongFrameColumnWriter implements FrameColumnWriter
|
|||
)
|
||||
{
|
||||
this.selector = selector;
|
||||
this.appendableMemory = AppendableMemory.create(allocator);
|
||||
this.hasNulls = hasNulls;
|
||||
this.sz = valueSize(hasNulls);
|
||||
}
|
||||
|
||||
public static int valueSize(final boolean hasNulls)
|
||||
{
|
||||
return hasNulls ? Long.BYTES + 1 : Long.BYTES;
|
||||
this.maker = new LongFrameMaker(allocator, hasNulls);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean addSelection()
|
||||
{
|
||||
if (!(appendableMemory.reserveAdditional(sz))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final MemoryRange<WritableMemory> cursor = appendableMemory.cursor();
|
||||
final WritableMemory memory = cursor.memory();
|
||||
final long position = cursor.start();
|
||||
|
||||
if (hasNulls) {
|
||||
if (selector.isNull()) {
|
||||
memory.putByte(position, (byte) 1);
|
||||
memory.putLong(position + 1, 0);
|
||||
} else {
|
||||
memory.putByte(position, (byte) 0);
|
||||
memory.putLong(position + 1, selector.getLong());
|
||||
}
|
||||
if (selector.isNull()) {
|
||||
return maker.addNull();
|
||||
} else {
|
||||
memory.putLong(position, selector.getLong());
|
||||
return maker.add(selector.getLong());
|
||||
}
|
||||
|
||||
appendableMemory.advanceCursor(sz);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void undo()
|
||||
{
|
||||
appendableMemory.rewindCursor(sz);
|
||||
maker.undo();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size()
|
||||
{
|
||||
return DATA_OFFSET + appendableMemory.size();
|
||||
return maker.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long writeTo(final WritableMemory memory, final long startPosition)
|
||||
{
|
||||
long currentPosition = startPosition;
|
||||
|
||||
memory.putByte(currentPosition, FrameColumnWriters.TYPE_LONG);
|
||||
memory.putByte(currentPosition + 1, hasNulls ? (byte) 1 : (byte) 0);
|
||||
currentPosition += 2;
|
||||
|
||||
currentPosition += appendableMemory.writeTo(memory, currentPosition);
|
||||
return currentPosition - startPosition;
|
||||
return maker.writeTo(memory, startPosition);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
appendableMemory.close();
|
||||
maker.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,122 @@
|
|||
/*
|
||||
* 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.druid.frame.write.columnar;
|
||||
|
||||
import org.apache.datasketches.memory.WritableMemory;
|
||||
import org.apache.druid.frame.allocation.AppendableMemory;
|
||||
import org.apache.druid.frame.allocation.MemoryAllocator;
|
||||
import org.apache.druid.frame.allocation.MemoryRange;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
public class LongFrameMaker implements Closeable
|
||||
{
|
||||
public static final long DATA_OFFSET = 1 /* type code */ + 1 /* has nulls? */;
|
||||
|
||||
private final AppendableMemory appendableMemory;
|
||||
private final boolean hasNulls;
|
||||
private final int sz;
|
||||
|
||||
public LongFrameMaker(
|
||||
MemoryAllocator allocator,
|
||||
boolean hasNulls
|
||||
)
|
||||
{
|
||||
this.appendableMemory = AppendableMemory.create(allocator);
|
||||
this.hasNulls = hasNulls;
|
||||
this.sz = valueSize(hasNulls);
|
||||
}
|
||||
|
||||
public static int valueSize(final boolean hasNulls)
|
||||
{
|
||||
return hasNulls ? Long.BYTES + 1 : Long.BYTES;
|
||||
}
|
||||
|
||||
public boolean add(long value)
|
||||
{
|
||||
if (!(appendableMemory.reserveAdditional(sz))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final MemoryRange<WritableMemory> cursor = appendableMemory.cursor();
|
||||
final WritableMemory memory = cursor.memory();
|
||||
final long position = cursor.start();
|
||||
|
||||
if (hasNulls) {
|
||||
memory.putByte(position, (byte) 0);
|
||||
memory.putLong(position + 1, value);
|
||||
} else {
|
||||
memory.putLong(position, value);
|
||||
}
|
||||
|
||||
appendableMemory.advanceCursor(sz);
|
||||
return true;
|
||||
}
|
||||
|
||||
public boolean addNull()
|
||||
{
|
||||
if (!hasNulls) {
|
||||
throw new ISE("Was told that null doesn't exist, cannot add null");
|
||||
}
|
||||
|
||||
if (!(appendableMemory.reserveAdditional(sz))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final MemoryRange<WritableMemory> cursor = appendableMemory.cursor();
|
||||
final WritableMemory memory = cursor.memory();
|
||||
final long position = cursor.start();
|
||||
|
||||
memory.putByte(position, (byte) 1);
|
||||
memory.putLong(position + 1, 0);
|
||||
|
||||
appendableMemory.advanceCursor(sz);
|
||||
return true;
|
||||
}
|
||||
|
||||
public void undo()
|
||||
{
|
||||
appendableMemory.rewindCursor(sz);
|
||||
}
|
||||
|
||||
public long size()
|
||||
{
|
||||
return DATA_OFFSET + appendableMemory.size();
|
||||
}
|
||||
|
||||
public long writeTo(final WritableMemory memory, final long startPosition)
|
||||
{
|
||||
long currentPosition = startPosition;
|
||||
|
||||
memory.putByte(currentPosition, FrameColumnWriters.TYPE_LONG);
|
||||
memory.putByte(currentPosition + 1, hasNulls ? (byte) 1 : (byte) 0);
|
||||
currentPosition += 2;
|
||||
|
||||
currentPosition += appendableMemory.writeTo(memory, currentPosition);
|
||||
return currentPosition - startPosition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
appendableMemory.close();
|
||||
}
|
||||
}
|
|
@ -32,6 +32,7 @@ import org.apache.druid.segment.column.ColumnType;
|
|||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -70,7 +71,7 @@ public class InlineDataSource implements DataSource
|
|||
private static InlineDataSource fromJson(
|
||||
@JsonProperty("columnNames") List<String> columnNames,
|
||||
@JsonProperty("columnTypes") List<ColumnType> columnTypes,
|
||||
@JsonProperty("rows") List<Object[]> rows
|
||||
@JsonProperty("rows") ArrayList<Object[]> rows
|
||||
)
|
||||
{
|
||||
Preconditions.checkNotNull(columnNames, "'columnNames' must be nonnull");
|
||||
|
@ -200,6 +201,11 @@ public class InlineDataSource implements DataSource
|
|||
return rows;
|
||||
}
|
||||
|
||||
public boolean rowsAreArrayList()
|
||||
{
|
||||
return rows instanceof ArrayList;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<DataSource> getChildren()
|
||||
{
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.druid.query.datasourcemetadata.DataSourceMetadataQuery;
|
|||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||
import org.apache.druid.query.operator.WindowOperatorQuery;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
import org.apache.druid.query.search.SearchQuery;
|
||||
import org.apache.druid.query.select.SelectQuery;
|
||||
|
@ -45,7 +46,6 @@ import org.joda.time.Duration;
|
|||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -54,27 +54,29 @@ import java.util.UUID;
|
|||
@ExtensionPoint
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "queryType")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = Query.TIMESERIES, value = TimeseriesQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.SEARCH, value = SearchQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.DATASOURCE_METADATA, value = DataSourceMetadataQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.GROUP_BY, value = GroupByQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.SCAN, value = ScanQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.SEARCH, value = SearchQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.SEGMENT_METADATA, value = SegmentMetadataQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.SELECT, value = SelectQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.TIMESERIES, value = TimeseriesQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.TOPN, value = TopNQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.DATASOURCE_METADATA, value = DataSourceMetadataQuery.class)
|
||||
@JsonSubTypes.Type(name = Query.WINDOW_OPERATOR, value = WindowOperatorQuery.class),
|
||||
})
|
||||
public interface Query<T>
|
||||
{
|
||||
String TIMESERIES = "timeseries";
|
||||
String SEARCH = "search";
|
||||
String TIME_BOUNDARY = "timeBoundary";
|
||||
String DATASOURCE_METADATA = "dataSourceMetadata";
|
||||
String GROUP_BY = "groupBy";
|
||||
String SCAN = "scan";
|
||||
String SEARCH = "search";
|
||||
String SEGMENT_METADATA = "segmentMetadata";
|
||||
String SELECT = "select";
|
||||
String TIME_BOUNDARY = "timeBoundary";
|
||||
String TIMESERIES = "timeseries";
|
||||
String TOPN = "topN";
|
||||
String DATASOURCE_METADATA = "dataSourceMetadata";
|
||||
String WINDOW_OPERATOR = "windowOperator";
|
||||
|
||||
DataSource getDataSource();
|
||||
|
||||
|
|
|
@ -0,0 +1,115 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.operator.window.value.ShiftedColumnAccessorBase;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collection;
|
||||
|
||||
public class LimitedRowsAndColumns implements RowsAndColumns
|
||||
{
|
||||
private final RowsAndColumns rac;
|
||||
private final int start;
|
||||
private final int end;
|
||||
|
||||
public LimitedRowsAndColumns(RowsAndColumns rac, int start, int end)
|
||||
{
|
||||
final int numRows = rac.numRows();
|
||||
if (numRows < end) {
|
||||
throw new ISE("end[%d] is out of bounds, cannot be greater than numRows[%d]", end, numRows);
|
||||
}
|
||||
|
||||
this.rac = rac;
|
||||
this.start = start;
|
||||
this.end = end;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> getColumnNames()
|
||||
{
|
||||
return rac.getColumnNames();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return end - start;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Column findColumn(String name)
|
||||
{
|
||||
final Column column = rac.findColumn(name);
|
||||
if (column == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return new Column()
|
||||
{
|
||||
@Override
|
||||
public ColumnAccessor toAccessor()
|
||||
{
|
||||
final ColumnAccessor columnAccessor = column.toAccessor();
|
||||
return new ShiftedColumnAccessorBase(columnAccessor)
|
||||
{
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return end - start;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int getActualCell(int cell)
|
||||
{
|
||||
int retVal = start + cell;
|
||||
if (retVal >= end) {
|
||||
throw new ISE("Index out of bounds[%d] >= [%d], start[%s]", retVal, end, start);
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean outsideBounds(int cell)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T as(Class<? extends T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public <T> T as(Class<T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,100 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.rowsandcols.DefaultSortedGroupPartitioner;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.SortedGroupPartitioner;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* This naive partitioning operator assumes that it's child operator always gives it RowsAndColumns objects that are
|
||||
* a superset of the partitions that it needs to provide. It will never attempt to make a partition larger than a
|
||||
* single RowsAndColumns object that it is given from its child Operator. A different operator should be used
|
||||
* if that is an important bit of functionality to have.
|
||||
* <p>
|
||||
* Additionally, this assumes that data has been pre-sorted according to the partitioning columns. If it is
|
||||
* given data that has not been pre-sorted, an exception is expected to be thrown.
|
||||
*/
|
||||
public class NaivePartitioningOperator implements Operator
|
||||
{
|
||||
private final List<String> partitionColumns;
|
||||
private final Operator child;
|
||||
|
||||
private Iterator<RowsAndColumns> partitionsIter;
|
||||
|
||||
public NaivePartitioningOperator(
|
||||
List<String> partitionColumns,
|
||||
Operator child
|
||||
)
|
||||
{
|
||||
this.partitionColumns = partitionColumns;
|
||||
this.child = child;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open()
|
||||
{
|
||||
child.open();
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns next()
|
||||
{
|
||||
if (partitionsIter != null && partitionsIter.hasNext()) {
|
||||
return partitionsIter.next();
|
||||
}
|
||||
|
||||
if (child.hasNext()) {
|
||||
final RowsAndColumns rac = child.next();
|
||||
|
||||
SortedGroupPartitioner groupPartitioner = rac.as(SortedGroupPartitioner.class);
|
||||
if (groupPartitioner == null) {
|
||||
groupPartitioner = new DefaultSortedGroupPartitioner(rac);
|
||||
}
|
||||
|
||||
partitionsIter = groupPartitioner.partitionOnBoundaries(partitionColumns).iterator();
|
||||
return partitionsIter.next();
|
||||
}
|
||||
|
||||
throw new ISE("Asked for next when already complete");
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
if (partitionsIter != null && partitionsIter.hasNext()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
return child.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(boolean cascade)
|
||||
{
|
||||
if (cascade) {
|
||||
child.close(cascade);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class NaivePartitioningOperatorFactory implements OperatorFactory
|
||||
{
|
||||
private final List<String> partitionColumns;
|
||||
|
||||
@JsonCreator
|
||||
public NaivePartitioningOperatorFactory(
|
||||
@JsonProperty("partitionColumns") List<String> partitionColumns
|
||||
)
|
||||
{
|
||||
this.partitionColumns = partitionColumns == null ? new ArrayList<>() : partitionColumns;
|
||||
}
|
||||
|
||||
@JsonProperty("partitionColumns")
|
||||
public List<String> getPartitionColumns()
|
||||
{
|
||||
return partitionColumns;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Operator wrap(Operator op)
|
||||
{
|
||||
return new NaivePartitioningOperator(partitionColumns, op);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(OperatorFactory other)
|
||||
{
|
||||
if (other instanceof NaivePartitioningOperatorFactory) {
|
||||
return partitionColumns.equals(((NaivePartitioningOperatorFactory) other).getPartitionColumns());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "NaivePartitioningOperatorFactory{" +
|
||||
"partitionColumns=" + partitionColumns +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,93 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
|
||||
/**
|
||||
* An Operator interface that intends to align closely with the Operators that other databases would also tend
|
||||
* to be implemented using.
|
||||
* <p>
|
||||
* The lifecycle of an operator is that, after creation, it should be opened, and then iterated using hasNext() and
|
||||
* next(). Finally, when the Operator is no longer useful, it should be closed.
|
||||
* <p>
|
||||
* Operator's methods mimic the methods of an {@code Iterator}, but it does not implement {@code Iterator}
|
||||
* intentionally. An operator should never be wrapped in an {@code Iterator}. Any code that does that should be
|
||||
* considered a bug and fixed. This is for two reasons:
|
||||
* <p>
|
||||
* 1. An Operator should never be passed around as an {@code Iterator}. An Operator must be closed, if an operator
|
||||
* gets returned as an {@code Iterator}, the code that sees the {@code Iterator} loses the knowledge that it's
|
||||
* dealing with an Operator and might not close it. Even something like a {@code CloseableIterator} is an
|
||||
* anti-pattern as it's possible to use it in a functional manner with code that loses track of the fact that it
|
||||
* must be closed.
|
||||
* 2. To avoid "fluent" style composition of functions on Operators. It is important that there never be a set of
|
||||
* functional primitives for things like map/filter/reduce to "simplify" the implementation of Operators. This is
|
||||
* because such fluency produces really hard to decipher stack traces as the stacktrace ends up being just a bunch
|
||||
* of calls from the scaffolding (map/filter/reduce) and not from the actual Operator itself. By not implementing
|
||||
* {@code Iterator} we are actively increasing the burden of trying to add such functional operations to the point
|
||||
* that hopefully, though code review, we can ensure that we never develop them. It is infinitely better to preserve
|
||||
* the stacktrace and "duplicate" the map/filter/reduce scaffolding code.
|
||||
*/
|
||||
public interface Operator
|
||||
{
|
||||
/**
|
||||
* Called to initiate the lifecycle of the Operator. If an operator needs to checkout resources or anything to do
|
||||
* its work, this is probably the place to do it.
|
||||
*
|
||||
* Work should *never* be done in this method, this method only exists to acquire resources that are known to be
|
||||
* needed before doing any work. As a litmus test, if there is ever a call to `op.next()` inside of this method,
|
||||
* then something has been done wrong as that call to `.next()` is actually doing work. Such code should be moved
|
||||
* into being lazily evaluated as part of a call to `.next()`.
|
||||
*/
|
||||
void open();
|
||||
|
||||
/**
|
||||
* Returns the next RowsAndColumns object that the Operator can produce. Behavior is undefined if
|
||||
* {@link #hasNext} returns false.
|
||||
*
|
||||
* @return the next RowsAndColumns object that the operator can produce
|
||||
*/
|
||||
RowsAndColumns next();
|
||||
|
||||
/**
|
||||
* Used to identify if it is safe to call {@link #next}
|
||||
*
|
||||
* @return true if it is safe to call {@link #next}
|
||||
*/
|
||||
boolean hasNext();
|
||||
|
||||
/**
|
||||
* Closes this Operator. The cascade flag can be used to identify that the intent is to close this operator
|
||||
* and only this operator without actually closing child operators. Other databases us this sort of functionality
|
||||
* with a planner that is watching over all of the objects and force-closes even if they were closed during normal
|
||||
* operations. In Druid, in the data pipeline where this was introduced, we are guaranteed to always have close
|
||||
* called regardless of errors or exceptions during processing, as such, at time of introduction, there is no
|
||||
* call that passes false for cascade.
|
||||
* <p>
|
||||
* That said, given that this is a common thing for these interfaces for other databases, we want to preserve the
|
||||
* optionality of being able to leverage what they do. As such, we define the method this way with the belief
|
||||
* that it might be used in the future. Semantically, this means that all implementations of Operators must
|
||||
* expect to be closed multiple times. I.e. after being closed, it is an error for open, next or hasNext to be
|
||||
* called, but close can be called any number of times.
|
||||
*
|
||||
* @param cascade whether to call close on child operators.
|
||||
*/
|
||||
void close(boolean cascade);
|
||||
}
|
|
@ -0,0 +1,59 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
/**
|
||||
* A factory for Operators. This class exists to encapsulate the user-definition of an Operator. I.e. which operator,
|
||||
* what fields it should operate on, etc. etc. These Factory objects are then used to combine Operators together
|
||||
* and run against concrete data.
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "naivePartition", value = NaivePartitioningOperatorFactory.class),
|
||||
@JsonSubTypes.Type(name = "window", value = WindowOperatorFactory.class),
|
||||
})
|
||||
public interface OperatorFactory
|
||||
{
|
||||
/**
|
||||
* Builds an operator according to the definition of the OperatorFactory and wraps it around the operator passed
|
||||
* in to this function.
|
||||
*
|
||||
* @param op the Operator to wrap
|
||||
* @return the wrapped Operator
|
||||
*/
|
||||
Operator wrap(Operator op);
|
||||
|
||||
/**
|
||||
* Validates the equivalence of Operators. This is similar to @{code .equals} but is its own method
|
||||
* so that it can ignore certain fields that would be important for a true equality check. Namely, two Operators
|
||||
* defined the same way but with different output names can be considered equivalent even though they are not equal.
|
||||
* <p>
|
||||
* This primarily exists to simplify tests, where this equivalence can be used to validate that the Operators
|
||||
* created by the SQL planner are actually equivalent to what we expect without needing to be overly dependent on
|
||||
* how the planner names output columns
|
||||
*
|
||||
* @param other the processor to test equivalence of
|
||||
* @return boolean identifying if these processors should be considered equivalent to each other.
|
||||
*/
|
||||
boolean validateEquivalent(OperatorFactory other);
|
||||
}
|
|
@ -0,0 +1,125 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import org.apache.druid.java.util.common.guava.Accumulator;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.java.util.common.guava.Yielder;
|
||||
import org.apache.druid.java.util.common.guava.Yielders;
|
||||
import org.apache.druid.java.util.common.guava.YieldingAccumulator;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public class OperatorSequence implements Sequence<RowsAndColumns>
|
||||
{
|
||||
private final Supplier<Operator> opSupplier;
|
||||
|
||||
public OperatorSequence(
|
||||
Supplier<Operator> opSupplier
|
||||
)
|
||||
{
|
||||
this.opSupplier = opSupplier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <OutType> OutType accumulate(
|
||||
OutType initValue,
|
||||
Accumulator<OutType, RowsAndColumns> accumulator
|
||||
)
|
||||
{
|
||||
Operator op = null;
|
||||
try {
|
||||
op = opSupplier.get();
|
||||
op.open();
|
||||
while (op.hasNext()) {
|
||||
initValue = accumulator.accumulate(initValue, op.next());
|
||||
}
|
||||
return initValue;
|
||||
}
|
||||
finally {
|
||||
if (op != null) {
|
||||
op.close(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <OutType> Yielder<OutType> toYielder(
|
||||
OutType initValue,
|
||||
YieldingAccumulator<OutType, RowsAndColumns> accumulator
|
||||
)
|
||||
{
|
||||
final Operator op = opSupplier.get();
|
||||
try {
|
||||
op.open();
|
||||
|
||||
while (!accumulator.yielded() && op.hasNext()) {
|
||||
initValue = accumulator.accumulate(initValue, op.next());
|
||||
}
|
||||
if (accumulator.yielded()) {
|
||||
OutType finalInitValue = initValue;
|
||||
return new Yielder<OutType>()
|
||||
{
|
||||
private OutType retVal = finalInitValue;
|
||||
private boolean done = false;
|
||||
|
||||
@Override
|
||||
public OutType get()
|
||||
{
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Yielder<OutType> next(OutType initValue)
|
||||
{
|
||||
accumulator.reset();
|
||||
retVal = initValue;
|
||||
while (!accumulator.yielded() && op.hasNext()) {
|
||||
retVal = accumulator.accumulate(retVal, op.next());
|
||||
}
|
||||
if (!accumulator.yielded()) {
|
||||
done = true;
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isDone()
|
||||
{
|
||||
return done;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
op.close(true);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return Yielders.done(initValue, () -> op.close(true));
|
||||
}
|
||||
}
|
||||
catch (RuntimeException e) {
|
||||
op.close(true);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.segment.Segment;
|
||||
|
||||
public class SegmentToRowsAndColumnsOperator implements Operator
|
||||
{
|
||||
private final Segment segment;
|
||||
private boolean hasNext = true;
|
||||
|
||||
public SegmentToRowsAndColumnsOperator(
|
||||
Segment segment
|
||||
)
|
||||
{
|
||||
this.segment = segment;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open()
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns next()
|
||||
{
|
||||
hasNext = false;
|
||||
|
||||
RowsAndColumns rac = segment.as(RowsAndColumns.class);
|
||||
if (rac != null) {
|
||||
return rac;
|
||||
}
|
||||
|
||||
throw new ISE("Cannot work with segment of type[%s]", segment.getClass());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
return hasNext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(boolean cascade)
|
||||
{
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,87 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.java.util.common.guava.Yielder;
|
||||
import org.apache.druid.java.util.common.guava.Yielders;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
public class SequenceOperator implements Operator
|
||||
{
|
||||
private final Sequence<RowsAndColumns> child;
|
||||
private Yielder<RowsAndColumns> yielder;
|
||||
private boolean closed = false;
|
||||
|
||||
public SequenceOperator(
|
||||
Sequence<RowsAndColumns> child
|
||||
)
|
||||
{
|
||||
this.child = child;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open()
|
||||
{
|
||||
if (closed) {
|
||||
throw new ISE("Operator closed, cannot be re-opened");
|
||||
}
|
||||
yielder = Yielders.each(child);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns next()
|
||||
{
|
||||
if (closed) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
final RowsAndColumns retVal = yielder.get();
|
||||
yielder = yielder.next(null);
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
return !closed && !yielder.isDone();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(boolean cascade)
|
||||
{
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
try {
|
||||
yielder.close();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RE(e, "Exception when closing yielder from Sequence");
|
||||
}
|
||||
finally {
|
||||
closed = true;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
|
||||
public class WindowOperatorFactory implements OperatorFactory
|
||||
{
|
||||
private Processor processor;
|
||||
|
||||
@JsonCreator
|
||||
public WindowOperatorFactory(
|
||||
@JsonProperty("processor") Processor processor
|
||||
)
|
||||
{
|
||||
Preconditions.checkNotNull(processor, "processor cannot be null");
|
||||
this.processor = processor;
|
||||
}
|
||||
|
||||
@JsonProperty("processor")
|
||||
public Processor getProcessor()
|
||||
{
|
||||
return processor;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Operator wrap(Operator op)
|
||||
{
|
||||
return new WindowProcessorOperator(processor, op);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(OperatorFactory other)
|
||||
{
|
||||
if (other instanceof WindowOperatorFactory) {
|
||||
return processor.validateEquivalent(((WindowOperatorFactory) other).getProcessor());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "WindowOperatorFactory{" +
|
||||
"processor=" + processor +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,167 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.query.BaseQuery;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.spec.LegacySegmentSpec;
|
||||
import org.apache.druid.query.spec.QuerySegmentSpec;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A query that can compute window functions on top of a completely in-memory inline datasource or query results.
|
||||
* <p>
|
||||
* It relies on a set of Operators to work on the data that it is given. As such, it doesn't actually encapsulate
|
||||
* any window-specific logic in-and-of-itself, but rather delegates everything to the operators. This is because
|
||||
* this is also intended as the initial addition of more explicit Operators to the Druid code base.
|
||||
* <p>
|
||||
* The assumptions on the incoming data are defined by the operators. At initial time of writing, there is a baked
|
||||
* in assumption that data has been sorted "correctly" before this runs.
|
||||
*/
|
||||
public class WindowOperatorQuery extends BaseQuery<RowsAndColumns>
|
||||
{
|
||||
private final RowSignature rowSignature;
|
||||
private final List<OperatorFactory> operators;
|
||||
|
||||
@JsonCreator
|
||||
public WindowOperatorQuery(
|
||||
@JsonProperty("dataSource") DataSource dataSource,
|
||||
@JsonProperty("context") Map<String, Object> context,
|
||||
@JsonProperty("outputSignature") RowSignature rowSignature,
|
||||
@JsonProperty("operatorDefinition") List<OperatorFactory> operators
|
||||
)
|
||||
{
|
||||
super(dataSource, new LegacySegmentSpec(Intervals.ETERNITY), false, context);
|
||||
this.rowSignature = rowSignature;
|
||||
this.operators = operators;
|
||||
if (!(dataSource instanceof QueryDataSource || dataSource instanceof InlineDataSource)) {
|
||||
throw new IAE("WindowOperatorQuery must run on top of a query or inline data source, got [%s]", dataSource);
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty("operatorDefinition")
|
||||
public List<OperatorFactory> getOperators()
|
||||
{
|
||||
return operators;
|
||||
}
|
||||
|
||||
@JsonProperty("outputSignature")
|
||||
public RowSignature getRowSignature()
|
||||
{
|
||||
return rowSignature;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasFilters()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimFilter getFilter()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return Query.WINDOW_OPERATOR;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query<RowsAndColumns> withOverriddenContext(Map<String, Object> contextOverride)
|
||||
{
|
||||
return new WindowOperatorQuery(
|
||||
getDataSource(),
|
||||
computeOverriddenContext(getContext(), contextOverride),
|
||||
rowSignature,
|
||||
operators
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query<RowsAndColumns> withQuerySegmentSpec(QuerySegmentSpec spec)
|
||||
{
|
||||
throw new UOE("Cannot override querySegmentSpec on window operator query. [%s]", spec);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query<RowsAndColumns> withDataSource(DataSource dataSource)
|
||||
{
|
||||
return new WindowOperatorQuery(
|
||||
dataSource,
|
||||
getContext(),
|
||||
rowSignature,
|
||||
operators
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
WindowOperatorQuery that = (WindowOperatorQuery) o;
|
||||
return Objects.equals(rowSignature, that.rowSignature) && Objects.equals(
|
||||
operators,
|
||||
that.operators
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), rowSignature, operators);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "WindowOperatorQuery{" +
|
||||
"dataSource='" + getDataSource() + '\'' +
|
||||
", querySegmentSpec=" + getQuerySegmentSpec() +
|
||||
", context=" + getContext() +
|
||||
", rowSignature=" + rowSignature +
|
||||
", operators=" + operators +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.operator;
|
||||
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.query.QueryProcessingPool;
|
||||
import org.apache.druid.query.QueryRunner;
|
||||
import org.apache.druid.query.QueryRunnerFactory;
|
||||
import org.apache.druid.query.QueryToolChest;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.segment.Segment;
|
||||
|
||||
public class WindowOperatorQueryQueryRunnerFactory implements QueryRunnerFactory<RowsAndColumns, WindowOperatorQuery>
|
||||
{
|
||||
public static final WindowOperatorQueryQueryToolChest TOOLCHEST = new WindowOperatorQueryQueryToolChest();
|
||||
|
||||
@Override
|
||||
public QueryRunner<RowsAndColumns> createRunner(Segment segment)
|
||||
{
|
||||
return (queryPlus, responseContext) ->
|
||||
new OperatorSequence(() -> new SegmentToRowsAndColumnsOperator(segment));
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner<RowsAndColumns> mergeRunners(
|
||||
QueryProcessingPool queryProcessingPool,
|
||||
Iterable<QueryRunner<RowsAndColumns>> queryRunners
|
||||
)
|
||||
{
|
||||
return Iterables.getOnlyElement(queryRunners);
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryToolChest<RowsAndColumns, WindowOperatorQuery> getToolchest()
|
||||
{
|
||||
return TOOLCHEST;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,191 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Functions;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.query.DefaultQueryMetrics;
|
||||
import org.apache.druid.query.QueryMetrics;
|
||||
import org.apache.druid.query.QueryPlus;
|
||||
import org.apache.druid.query.QueryRunner;
|
||||
import org.apache.druid.query.QueryToolChest;
|
||||
import org.apache.druid.query.aggregation.MetricManipulationFn;
|
||||
import org.apache.druid.query.context.ResponseContext;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
import org.apache.druid.query.rowsandcols.column.NullColumnAccessor;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public class WindowOperatorQueryQueryToolChest extends QueryToolChest<RowsAndColumns, WindowOperatorQuery>
|
||||
{
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public QueryRunner<RowsAndColumns> mergeResults(QueryRunner<RowsAndColumns> runner)
|
||||
{
|
||||
return new RowsAndColumnsUnravelingQueryRunner(
|
||||
(queryPlus, responseContext) -> {
|
||||
final WindowOperatorQuery query = (WindowOperatorQuery) queryPlus.getQuery();
|
||||
final List<OperatorFactory> opFactories = query.getOperators();
|
||||
|
||||
Supplier<Operator> opSupplier = () -> {
|
||||
Operator retVal = new SequenceOperator(runner.run(queryPlus, responseContext));
|
||||
for (OperatorFactory operatorFactory : opFactories) {
|
||||
retVal = operatorFactory.wrap(retVal);
|
||||
}
|
||||
return retVal;
|
||||
};
|
||||
|
||||
return new OperatorSequence(opSupplier);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryMetrics<? super WindowOperatorQuery> makeMetrics(WindowOperatorQuery query)
|
||||
{
|
||||
return new DefaultQueryMetrics<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<RowsAndColumns, RowsAndColumns> makePreComputeManipulatorFn(
|
||||
WindowOperatorQuery query,
|
||||
MetricManipulationFn fn
|
||||
)
|
||||
{
|
||||
return Functions.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<RowsAndColumns> getResultTypeReference()
|
||||
{
|
||||
return new TypeReference<RowsAndColumns>()
|
||||
{
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowSignature resultArraySignature(WindowOperatorQuery query)
|
||||
{
|
||||
return query.getRowSignature();
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public Sequence<Object[]> resultsAsArrays(
|
||||
WindowOperatorQuery query,
|
||||
Sequence<RowsAndColumns> resultSequence
|
||||
)
|
||||
{
|
||||
// Dark magic; see RowsAndColumnsUnravelingQueryRunner.
|
||||
return (Sequence) resultSequence;
|
||||
}
|
||||
|
||||
/**
|
||||
* This class exists to unravel the RowsAndColumns that are used in this query and make it the return Sequence
|
||||
* actually be a Sequence of rows. This is relatively broken in a number of regards, the most obvious of which
|
||||
* is that it is going to run counter to the stated class on the Generic of the QueryToolChest. That is, the
|
||||
* code makes it look like you are getting a Sequence of RowsAndColumns, but, by using this, the query will
|
||||
* actually ultimately produce a Sequence of Object[]. This works because of type Erasure in Java (it's all Object
|
||||
* at the end of the day).
|
||||
* <p>
|
||||
* While it might seem like this will break all sorts of things, the Generic type is actually there more as a type
|
||||
* "hint" to make the writing of the ToolChest and Factory and stuff a bit more simple. Any caller of this cannot
|
||||
* truly depend on the type anyway other than to just throw it across the wire, so this should just magically work
|
||||
* even though it looks like it shouldn't even compile.
|
||||
* <p>
|
||||
* Not our proudest moment, but we use the tools available to us.
|
||||
*/
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
private static class RowsAndColumnsUnravelingQueryRunner implements QueryRunner
|
||||
{
|
||||
|
||||
private final QueryRunner<RowsAndColumns> baseQueryRunner;
|
||||
|
||||
private RowsAndColumnsUnravelingQueryRunner(
|
||||
QueryRunner<RowsAndColumns> baseQueryRunner
|
||||
)
|
||||
{
|
||||
this.baseQueryRunner = baseQueryRunner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence run(
|
||||
QueryPlus queryPlus,
|
||||
ResponseContext responseContext
|
||||
)
|
||||
{
|
||||
// We only want to do this operation once at the very, very top of the execution tree. So we check and set
|
||||
// a context parameter so that if this merge code runs anywhere else, it will skip this part.
|
||||
final WindowOperatorQuery query = (WindowOperatorQuery) queryPlus.getQuery();
|
||||
if (query.context().getBoolean("unravel", true)) {
|
||||
final Sequence<RowsAndColumns> baseSequence = baseQueryRunner.run(
|
||||
queryPlus.withQuery(query.withOverriddenContext(ImmutableMap.of("unravel", false))),
|
||||
responseContext
|
||||
);
|
||||
|
||||
final RowSignature rowSignature = query.getRowSignature();
|
||||
return baseSequence.flatMap(
|
||||
rac -> {
|
||||
List<Object[]> results = new ArrayList<>(rac.numRows());
|
||||
|
||||
ColumnAccessor[] accessors = new ColumnAccessor[rowSignature.size()];
|
||||
int index = 0;
|
||||
for (String columnName : rowSignature.getColumnNames()) {
|
||||
final Column column = rac.findColumn(columnName);
|
||||
if (column == null) {
|
||||
final ColumnType columnType = rowSignature
|
||||
.getColumnType(columnName)
|
||||
.orElse(ColumnType.UNKNOWN_COMPLEX);
|
||||
|
||||
accessors[index] = new NullColumnAccessor(columnType, rac.numRows());
|
||||
} else {
|
||||
accessors[index] = column.toAccessor();
|
||||
}
|
||||
++index;
|
||||
}
|
||||
|
||||
for (int i = 0; i < rac.numRows(); ++i) {
|
||||
Object[] objArr = new Object[accessors.length];
|
||||
for (int j = 0; j < accessors.length; j++) {
|
||||
objArr[j] = accessors[j].getObject(i);
|
||||
}
|
||||
results.add(objArr);
|
||||
}
|
||||
|
||||
return Sequences.simple(results);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
return baseQueryRunner.run(queryPlus, responseContext);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,67 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
|
||||
/**
|
||||
* An Operator that applies a {@link Processor}, see javadoc on that interface for an explanation.
|
||||
*/
|
||||
public class WindowProcessorOperator implements Operator
|
||||
{
|
||||
private final Processor windowProcessor;
|
||||
private final Operator child;
|
||||
|
||||
public WindowProcessorOperator(
|
||||
Processor windowProcessor,
|
||||
Operator child
|
||||
)
|
||||
{
|
||||
this.windowProcessor = windowProcessor;
|
||||
this.child = child;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open()
|
||||
{
|
||||
child.open();
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns next()
|
||||
{
|
||||
return windowProcessor.process(child.next());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
return child.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(boolean cascade)
|
||||
{
|
||||
if (cascade) {
|
||||
child.close(cascade);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class ComposingProcessor implements Processor
|
||||
{
|
||||
private final Processor[] processors;
|
||||
|
||||
@JsonCreator
|
||||
public ComposingProcessor(
|
||||
@JsonProperty("processors") Processor... processors
|
||||
)
|
||||
{
|
||||
this.processors = processors;
|
||||
}
|
||||
|
||||
@JsonProperty("processors")
|
||||
public Processor[] getProcessors()
|
||||
{
|
||||
return processors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns incomingPartition)
|
||||
{
|
||||
RowsAndColumns retVal = incomingPartition;
|
||||
for (int i = processors.length - 1; i >= 0; --i) {
|
||||
retVal = processors[i].process(retVal);
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(Processor otherProcessor)
|
||||
{
|
||||
if (otherProcessor instanceof ComposingProcessor) {
|
||||
ComposingProcessor other = (ComposingProcessor) otherProcessor;
|
||||
for (int i = 0; i < processors.length; ++i) {
|
||||
if (!processors[i].validateEquivalent(other.processors[i])) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ComposingProcessor{" +
|
||||
"processors=" + Arrays.toString(processors) +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,83 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import org.apache.druid.query.operator.window.ranking.WindowCumeDistProcessor;
|
||||
import org.apache.druid.query.operator.window.ranking.WindowDenseRankProcessor;
|
||||
import org.apache.druid.query.operator.window.ranking.WindowPercentileProcessor;
|
||||
import org.apache.druid.query.operator.window.ranking.WindowRankProcessor;
|
||||
import org.apache.druid.query.operator.window.ranking.WindowRowNumberProcessor;
|
||||
import org.apache.druid.query.operator.window.value.WindowFirstProcessor;
|
||||
import org.apache.druid.query.operator.window.value.WindowLastProcessor;
|
||||
import org.apache.druid.query.operator.window.value.WindowOffsetProcessor;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
|
||||
/**
|
||||
* A Processor is a bit of logic that processes a single RowsAndColumns object to produce a new RowsAndColumns
|
||||
* object. Generally speaking, it is used to add or alter columns in a batch-oriented fashion.
|
||||
* <p>
|
||||
* This interface was created to support windowing functions, where the windowing function can be implemented
|
||||
* assuming that each RowsAndColumns object represents one partition. Thus, the window function implementation
|
||||
* can only need to worry about how to process a single partition at a time and something external to the window
|
||||
* function worries about providing data with the correct partitioning.
|
||||
* <p>
|
||||
* Over time, it's possible that this interface is used for other purposes as well, but the fundamental idea of
|
||||
* usages of the interface should always be doing a one-to-one transformation of RowsAndColumns objects. That is,
|
||||
* it's a RowsAndColumns in and a RowsAndColumns out.
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "composing", value = ComposingProcessor.class),
|
||||
@JsonSubTypes.Type(name = "cumeDist", value = WindowCumeDistProcessor.class),
|
||||
@JsonSubTypes.Type(name = "denseRank", value = WindowDenseRankProcessor.class),
|
||||
@JsonSubTypes.Type(name = "percentile", value = WindowPercentileProcessor.class),
|
||||
@JsonSubTypes.Type(name = "rank", value = WindowRankProcessor.class),
|
||||
@JsonSubTypes.Type(name = "rowNumber", value = WindowRowNumberProcessor.class),
|
||||
@JsonSubTypes.Type(name = "first", value = WindowFirstProcessor.class),
|
||||
@JsonSubTypes.Type(name = "last", value = WindowLastProcessor.class),
|
||||
@JsonSubTypes.Type(name = "offset", value = WindowOffsetProcessor.class),
|
||||
@JsonSubTypes.Type(name = "aggregate", value = WindowAggregateProcessor.class),
|
||||
})
|
||||
public interface Processor
|
||||
{
|
||||
/**
|
||||
* Applies the logic of the processor to a RowsAndColumns object
|
||||
*
|
||||
* @param incomingPartition the incoming RowsAndColumns object
|
||||
* @return the transformed RowsAndColumns object
|
||||
*/
|
||||
RowsAndColumns process(RowsAndColumns incomingPartition);
|
||||
|
||||
/**
|
||||
* Validates the equivalence of the Processors. This is similar to @{code .equals} but is its own method
|
||||
* so that it can ignore certain fields that would be important for a true equality check. Namely, two Processors
|
||||
* defined the same way but with different output names can be considered equivalent even though they are not equal.
|
||||
* <p>
|
||||
* This primarily exists to simplify tests, where this equivalence can be used to validate that the Processors
|
||||
* created by the SQL planner are actually equivalent to what we expect without needing to be overly dependent on
|
||||
* how the planner names the output columns
|
||||
*
|
||||
* @param otherProcessor the processor to test equivalence of
|
||||
* @return boolean identifying if these processors should be considered equivalent to each other.
|
||||
*/
|
||||
boolean validateEquivalent(Processor otherProcessor);
|
||||
}
|
|
@ -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.druid.query.operator.window;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.DefaultOnHeapAggregatable;
|
||||
import org.apache.druid.query.rowsandcols.OnHeapAggregatable;
|
||||
import org.apache.druid.query.rowsandcols.OnHeapCumulativeAggregatable;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.ConstantObjectColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public class WindowAggregateProcessor implements Processor
|
||||
{
|
||||
@Nullable
|
||||
private static <T> List<T> emptyToNull(List<T> list)
|
||||
{
|
||||
if (list == null || list.isEmpty()) {
|
||||
return null;
|
||||
} else {
|
||||
return list;
|
||||
}
|
||||
}
|
||||
|
||||
private final List<AggregatorFactory> aggregations;
|
||||
private final List<AggregatorFactory> cumulativeAggregations;
|
||||
|
||||
@JsonCreator
|
||||
public WindowAggregateProcessor(
|
||||
@JsonProperty("aggregations") List<AggregatorFactory> aggregations,
|
||||
@JsonProperty("cumulativeAggregations") List<AggregatorFactory> cumulativeAggregations
|
||||
)
|
||||
{
|
||||
this.aggregations = emptyToNull(aggregations);
|
||||
this.cumulativeAggregations = emptyToNull(cumulativeAggregations);
|
||||
}
|
||||
|
||||
@JsonProperty("aggregations")
|
||||
public List<AggregatorFactory> getAggregations()
|
||||
{
|
||||
return aggregations;
|
||||
}
|
||||
|
||||
@JsonProperty("cumulativeAggregations")
|
||||
public List<AggregatorFactory> getCumulativeAggregations()
|
||||
{
|
||||
return cumulativeAggregations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns inputPartition)
|
||||
{
|
||||
AppendableRowsAndColumns retVal = RowsAndColumns.expectAppendable(inputPartition);
|
||||
|
||||
if (aggregations != null) {
|
||||
OnHeapAggregatable aggregatable = inputPartition.as(OnHeapAggregatable.class);
|
||||
if (aggregatable == null) {
|
||||
aggregatable = new DefaultOnHeapAggregatable(inputPartition);
|
||||
}
|
||||
final ArrayList<Object> aggregatedVals = aggregatable.aggregateAll(aggregations);
|
||||
|
||||
for (int i = 0; i < aggregations.size(); ++i) {
|
||||
final AggregatorFactory agg = aggregations.get(i);
|
||||
retVal.addColumn(
|
||||
agg.getName(),
|
||||
new ConstantObjectColumn(aggregatedVals.get(i), inputPartition.numRows(), agg.getResultType())
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
if (cumulativeAggregations != null) {
|
||||
OnHeapCumulativeAggregatable cummulativeAgg = inputPartition.as(OnHeapCumulativeAggregatable.class);
|
||||
if (cummulativeAgg == null) {
|
||||
cummulativeAgg = new DefaultOnHeapAggregatable(inputPartition);
|
||||
}
|
||||
final ArrayList<Object[]> cumulativeVals = cummulativeAgg.aggregateCumulative(cumulativeAggregations);
|
||||
|
||||
for (int i = 0; i < cumulativeAggregations.size(); ++i) {
|
||||
final AggregatorFactory agg = cumulativeAggregations.get(i);
|
||||
retVal.addColumn(agg.getName(), new ObjectArrayColumn(cumulativeVals.get(i), agg.getResultType()));
|
||||
}
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(Processor otherProcessor)
|
||||
{
|
||||
if (otherProcessor instanceof WindowAggregateProcessor) {
|
||||
WindowAggregateProcessor other = (WindowAggregateProcessor) otherProcessor;
|
||||
return Objects.equals(aggregations, other.aggregations)
|
||||
&& Objects.equals(cumulativeAggregations, other.cumulativeAggregations);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "WindowAggregateProcessor{" +
|
||||
"aggregations=" + aggregations +
|
||||
", cumulativeAggregations=" + cumulativeAggregations +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.ranking;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* This Processor assumes that data has already been sorted for it. It does not re-sort the data and if it is given
|
||||
* data that is not in the correct sorted order, its operation is undefined.
|
||||
*/
|
||||
public class WindowCumeDistProcessor extends WindowRankingProcessorBase
|
||||
{
|
||||
@JsonCreator
|
||||
public WindowCumeDistProcessor(
|
||||
@JsonProperty("group") List<String> groupingCols,
|
||||
@JsonProperty("outputColumn") String outputColumn
|
||||
)
|
||||
{
|
||||
super(groupingCols, outputColumn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns incomingPartition)
|
||||
{
|
||||
return processInternal(incomingPartition, groupings -> {
|
||||
final double[] ranks = new double[incomingPartition.numRows()];
|
||||
for (int i = 1; i < groupings.length; ++i) {
|
||||
final int start = groupings[i - 1];
|
||||
final int end = groupings[i];
|
||||
double relativeRank = end / (double) ranks.length;
|
||||
Arrays.fill(ranks, start, end, relativeRank);
|
||||
}
|
||||
|
||||
return new DoubleArrayColumn(ranks);
|
||||
});
|
||||
}
|
||||
}
|
|
@ -0,0 +1,59 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.ranking;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* This Processor assumes that data has already been sorted for it. It does not re-sort the data and if it is given
|
||||
* data that is not in the correct sorted order, its operation is undefined.
|
||||
*/
|
||||
public class WindowDenseRankProcessor extends WindowRankingProcessorBase
|
||||
{
|
||||
@JsonCreator
|
||||
public WindowDenseRankProcessor(
|
||||
@JsonProperty("group") List<String> groupingCols,
|
||||
@JsonProperty("outputColumn") String outputColumn
|
||||
)
|
||||
{
|
||||
super(groupingCols, outputColumn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns incomingPartition)
|
||||
{
|
||||
return processInternal(incomingPartition, groupings -> {
|
||||
final int[] ranks = new int[incomingPartition.numRows()];
|
||||
for (int i = 1; i < groupings.length; ++i) {
|
||||
final int start = groupings[i - 1];
|
||||
final int end = groupings[i];
|
||||
Arrays.fill(ranks, start, end, i);
|
||||
}
|
||||
|
||||
return new IntArrayColumn(ranks);
|
||||
});
|
||||
}
|
||||
}
|
|
@ -0,0 +1,106 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.ranking;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class WindowPercentileProcessor implements Processor
|
||||
{
|
||||
private final int numBuckets;
|
||||
private final String outputColumn;
|
||||
|
||||
@JsonCreator
|
||||
public WindowPercentileProcessor(
|
||||
@JsonProperty("outputColumn") String outputColumn,
|
||||
@JsonProperty("numBuckets") int numBuckets
|
||||
)
|
||||
{
|
||||
Preconditions.checkArgument(numBuckets > 0, "numBuckets[%s] must be greater than zero", numBuckets);
|
||||
|
||||
this.outputColumn = outputColumn;
|
||||
this.numBuckets = numBuckets;
|
||||
}
|
||||
|
||||
@JsonProperty("numBuckets")
|
||||
public int getNumBuckets()
|
||||
{
|
||||
return numBuckets;
|
||||
}
|
||||
|
||||
@JsonProperty("outputColumn")
|
||||
public String getOutputColumn()
|
||||
{
|
||||
return outputColumn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns incomingPartition)
|
||||
{
|
||||
AppendableRowsAndColumns retVal = RowsAndColumns.expectAppendable(incomingPartition);
|
||||
|
||||
int numRows = incomingPartition.numRows();
|
||||
int countPerBucket = numRows / numBuckets;
|
||||
int extraRows = numRows % numBuckets;
|
||||
|
||||
int index = 0;
|
||||
int[] bucketVals = new int[numRows];
|
||||
for (int i = 0; i < numBuckets; ++i) {
|
||||
int nextIndex = index + countPerBucket;
|
||||
if (extraRows > 0) {
|
||||
++nextIndex;
|
||||
--extraRows;
|
||||
}
|
||||
|
||||
// Buckets are 1-indexed, so we fill with i+1
|
||||
Arrays.fill(bucketVals, index, nextIndex, i + 1);
|
||||
|
||||
index = nextIndex;
|
||||
}
|
||||
|
||||
retVal.addColumn(outputColumn, new IntArrayColumn(bucketVals));
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(Processor otherProcessor)
|
||||
{
|
||||
if (otherProcessor instanceof WindowPercentileProcessor) {
|
||||
return numBuckets == ((WindowPercentileProcessor) otherProcessor).numBuckets;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "WindowPercentileProcessor{" +
|
||||
"numBuckets=" + numBuckets +
|
||||
", outputColumn='" + outputColumn + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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.druid.query.operator.window.ranking;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* This Processor assumes that data has already been sorted for it. It does not re-sort the data and if it is given
|
||||
* data that is not in the correct sorted order, its operation is undefined.
|
||||
*/
|
||||
public class WindowRankProcessor extends WindowRankingProcessorBase
|
||||
{
|
||||
private final boolean asPercent;
|
||||
|
||||
@JsonCreator
|
||||
public WindowRankProcessor(
|
||||
@JsonProperty("group") List<String> groupingCols,
|
||||
@JsonProperty("outputColumn") String outputColumn,
|
||||
@JsonProperty("asPercent") boolean asPercent
|
||||
)
|
||||
{
|
||||
super(groupingCols, outputColumn);
|
||||
this.asPercent = asPercent;
|
||||
}
|
||||
|
||||
@JsonProperty("asPercent")
|
||||
public boolean isAsPercent()
|
||||
{
|
||||
return asPercent;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns incomingPartition)
|
||||
{
|
||||
if (asPercent) {
|
||||
return processInternal(incomingPartition, groupings -> {
|
||||
final double[] percentages = new double[incomingPartition.numRows()];
|
||||
if (percentages.length > 1) {
|
||||
final double denominator = percentages.length - 1;
|
||||
|
||||
for (int i = 1; i < groupings.length; ++i) {
|
||||
final int start = groupings[i - 1];
|
||||
final int end = groupings[i];
|
||||
Arrays.fill(percentages, start, end, start / denominator);
|
||||
}
|
||||
}
|
||||
|
||||
return new DoubleArrayColumn(percentages);
|
||||
});
|
||||
}
|
||||
|
||||
return processInternal(incomingPartition, groupings -> {
|
||||
final int[] ranks = new int[incomingPartition.numRows()];
|
||||
|
||||
for (int i = 1; i < groupings.length; ++i) {
|
||||
final int start = groupings[i - 1];
|
||||
final int end = groupings[i];
|
||||
Arrays.fill(ranks, start, end, start + 1);
|
||||
}
|
||||
|
||||
return new IntArrayColumn(ranks);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(Processor otherProcessor)
|
||||
{
|
||||
if (otherProcessor instanceof WindowRankProcessor) {
|
||||
WindowRankProcessor other = (WindowRankProcessor) otherProcessor;
|
||||
return asPercent == other.asPercent && intervalValidation(other);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "WindowRankProcessor{" +
|
||||
internalToString() +
|
||||
", asPercent=" + asPercent +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,103 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.ranking;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.DefaultSortedGroupPartitioner;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.SortedGroupPartitioner;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* This Processor assumes that data has already been sorted for it. It does not re-sort the data and if it is given
|
||||
* data that is not in the correct sorted order, its operation is undefined.
|
||||
*/
|
||||
public abstract class WindowRankingProcessorBase implements Processor
|
||||
{
|
||||
private final List<String> groupingCols;
|
||||
private final String outputColumn;
|
||||
|
||||
public WindowRankingProcessorBase(
|
||||
List<String> groupingCols,
|
||||
String outputColumn
|
||||
)
|
||||
{
|
||||
this.groupingCols = groupingCols;
|
||||
this.outputColumn = outputColumn;
|
||||
}
|
||||
|
||||
@JsonProperty("group")
|
||||
public List<String> getGroupingCols()
|
||||
{
|
||||
return groupingCols;
|
||||
}
|
||||
|
||||
@JsonProperty("outputColumn")
|
||||
public String getOutputColumn()
|
||||
{
|
||||
return outputColumn;
|
||||
}
|
||||
|
||||
public RowsAndColumns processInternal(
|
||||
RowsAndColumns incomingPartition,
|
||||
Function<int[], Column> fn
|
||||
)
|
||||
{
|
||||
final AppendableRowsAndColumns retVal = RowsAndColumns.expectAppendable(incomingPartition);
|
||||
|
||||
SortedGroupPartitioner groupPartitioner = incomingPartition.as(SortedGroupPartitioner.class);
|
||||
if (groupPartitioner == null) {
|
||||
groupPartitioner = new DefaultSortedGroupPartitioner(incomingPartition);
|
||||
}
|
||||
|
||||
retVal.addColumn(outputColumn, fn.apply(groupPartitioner.computeBoundaries(groupingCols)));
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(Processor otherProcessor)
|
||||
{
|
||||
return getClass() == otherProcessor.getClass()
|
||||
&& intervalValidation((WindowRankingProcessorBase) otherProcessor);
|
||||
}
|
||||
|
||||
protected boolean intervalValidation(WindowRankingProcessorBase other)
|
||||
{
|
||||
// Only input needs to be the same for the processors to produce equivalent results
|
||||
return groupingCols.equals(other.groupingCols);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return getClass().getSimpleName() + "{" + internalToString() + '}';
|
||||
}
|
||||
|
||||
protected String internalToString()
|
||||
{
|
||||
return "groupingCols=" + groupingCols +
|
||||
", outputColumn='" + outputColumn + '\'';
|
||||
}
|
||||
}
|
|
@ -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.druid.query.operator.window.ranking;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
public class WindowRowNumberProcessor implements Processor
|
||||
{
|
||||
private final String outputColumn;
|
||||
|
||||
@JsonCreator
|
||||
public WindowRowNumberProcessor(
|
||||
@JsonProperty("outputColumn") String outputColumn
|
||||
)
|
||||
{
|
||||
this.outputColumn = outputColumn;
|
||||
}
|
||||
|
||||
@JsonProperty("outputColumn")
|
||||
public String getOutputColumn()
|
||||
{
|
||||
return outputColumn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns incomingPartition)
|
||||
{
|
||||
final AppendableRowsAndColumns retVal = RowsAndColumns.expectAppendable(incomingPartition);
|
||||
retVal.addColumn(
|
||||
outputColumn,
|
||||
new ColumnAccessorBasedColumn(
|
||||
new ColumnAccessor()
|
||||
{
|
||||
@Override
|
||||
public ColumnType getType()
|
||||
{
|
||||
return ColumnType.LONG;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return incomingPartition.numRows();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull(int rowNum)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getObject(int rowNum)
|
||||
{
|
||||
return getInt(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble(int rowNum)
|
||||
{
|
||||
return getInt(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(int rowNum)
|
||||
{
|
||||
return getInt(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(int rowNum)
|
||||
{
|
||||
return getInt(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(int rowNum)
|
||||
{
|
||||
// cell is 0-indexed, rowNumbers are 1-indexed, so add 1.
|
||||
return rowNum + 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareCells(int lhsRowNum, int rhsRowNum)
|
||||
{
|
||||
return Integer.compare(lhsRowNum, rhsRowNum);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(Processor otherProcessor)
|
||||
{
|
||||
return otherProcessor instanceof WindowRowNumberProcessor;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "WindowRowNumberProcessor{" +
|
||||
"outputColumn='" + outputColumn + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,130 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.value;
|
||||
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
public abstract class ShiftedColumnAccessorBase implements ColumnAccessor
|
||||
{
|
||||
private final ColumnAccessor accessor;
|
||||
|
||||
public ShiftedColumnAccessorBase(ColumnAccessor accessor)
|
||||
{
|
||||
this.accessor = accessor;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnType getType()
|
||||
{
|
||||
return accessor.getType();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return accessor.numRows();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull(int rowNum)
|
||||
{
|
||||
final int actualCell = getActualCell(rowNum);
|
||||
if (outsideBounds(actualCell)) {
|
||||
return true;
|
||||
}
|
||||
return accessor.isNull(actualCell);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getObject(int rowNum)
|
||||
{
|
||||
final int actualCell = getActualCell(rowNum);
|
||||
if (outsideBounds(actualCell)) {
|
||||
return null;
|
||||
}
|
||||
return accessor.getObject(actualCell);
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble(int rowNum)
|
||||
{
|
||||
final int actualCell = getActualCell(rowNum);
|
||||
if (outsideBounds(actualCell)) {
|
||||
return 0.0D;
|
||||
}
|
||||
return accessor.getDouble(actualCell);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(int rowNum)
|
||||
{
|
||||
final int actualCell = getActualCell(rowNum);
|
||||
if (outsideBounds(actualCell)) {
|
||||
return 0.0F;
|
||||
}
|
||||
return accessor.getFloat(actualCell);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(int rowNum)
|
||||
{
|
||||
final int actualCell = getActualCell(rowNum);
|
||||
if (outsideBounds(actualCell)) {
|
||||
return 0L;
|
||||
}
|
||||
return accessor.getLong(actualCell);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(int rowNum)
|
||||
{
|
||||
final int actualCell = getActualCell(rowNum);
|
||||
if (outsideBounds(actualCell)) {
|
||||
return 0;
|
||||
}
|
||||
return accessor.getInt(actualCell);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareCells(int lhsRowNum, int rhsRowNum)
|
||||
{
|
||||
int actualLhsCell = getActualCell(lhsRowNum);
|
||||
int actualRhsCell = getActualCell(rhsRowNum);
|
||||
if (outsideBounds(actualLhsCell)) {
|
||||
if (outsideBounds(actualRhsCell)) {
|
||||
// Both are null
|
||||
return 0;
|
||||
} else {
|
||||
return accessor.isNull(actualRhsCell) ? 0 : -1;
|
||||
}
|
||||
} else {
|
||||
if (outsideBounds(actualRhsCell)) {
|
||||
return accessor.isNull(actualLhsCell) ? 0 : 1;
|
||||
} else {
|
||||
return accessor.compareCells(actualLhsCell, actualRhsCell);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract int getActualCell(int cell);
|
||||
|
||||
protected abstract boolean outsideBounds(int cell);
|
||||
}
|
|
@ -0,0 +1,50 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.value;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
import org.apache.druid.query.rowsandcols.column.ConstantObjectColumn;
|
||||
|
||||
public class WindowFirstProcessor extends WindowValueProcessorBase
|
||||
{
|
||||
@JsonCreator
|
||||
public WindowFirstProcessor(
|
||||
@JsonProperty("inputColumn") String inputColumn,
|
||||
@JsonProperty("outputColumn") String outputColumn
|
||||
)
|
||||
{
|
||||
super(inputColumn, outputColumn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns incomingPartition)
|
||||
{
|
||||
return processInternal(
|
||||
incomingPartition,
|
||||
column -> {
|
||||
final ColumnAccessor accessor = column.toAccessor();
|
||||
return new ConstantObjectColumn(accessor.getObject(0), accessor.numRows(), accessor.getType());
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.value;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
import org.apache.druid.query.rowsandcols.column.ConstantObjectColumn;
|
||||
|
||||
public class WindowLastProcessor extends WindowValueProcessorBase
|
||||
{
|
||||
@JsonCreator
|
||||
public WindowLastProcessor(
|
||||
@JsonProperty("inputColumn") String inputColumn,
|
||||
@JsonProperty("outputColumn") String outputColumn
|
||||
)
|
||||
{
|
||||
super(inputColumn, outputColumn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns input)
|
||||
{
|
||||
final int lastIndex = input.numRows() - 1;
|
||||
if (lastIndex < 0) {
|
||||
throw new ISE("Called with an input partition of size 0. The call site needs to not do that.");
|
||||
}
|
||||
|
||||
return processInternal(input, column -> {
|
||||
final ColumnAccessor accessor = column.toAccessor();
|
||||
return new ConstantObjectColumn(accessor.getObject(lastIndex), accessor.numRows(), accessor.getType());
|
||||
});
|
||||
}
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.value;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn;
|
||||
|
||||
public class WindowOffsetProcessor extends WindowValueProcessorBase
|
||||
{
|
||||
private final int offset;
|
||||
|
||||
@JsonCreator
|
||||
public WindowOffsetProcessor(
|
||||
@JsonProperty("inputColumn") String inputColumn,
|
||||
@JsonProperty("outputColumn") String outputColumn,
|
||||
@JsonProperty("offset") int offset
|
||||
)
|
||||
{
|
||||
super(inputColumn, outputColumn);
|
||||
this.offset = offset;
|
||||
}
|
||||
|
||||
@JsonProperty("offset")
|
||||
public int getOffset()
|
||||
{
|
||||
return offset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns input)
|
||||
{
|
||||
final int numRows = input.numRows();
|
||||
|
||||
return processInternal(input, column -> new ColumnAccessorBasedColumn(
|
||||
new ShiftedColumnAccessorBase(column.toAccessor())
|
||||
{
|
||||
@Override
|
||||
protected int getActualCell(int cell)
|
||||
{
|
||||
return cell + offset;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean outsideBounds(int actualCell)
|
||||
{
|
||||
return actualCell < 0 || actualCell >= numRows;
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(Processor otherProcessor)
|
||||
{
|
||||
if (otherProcessor instanceof WindowOffsetProcessor) {
|
||||
WindowOffsetProcessor other = (WindowOffsetProcessor) otherProcessor;
|
||||
return offset == other.offset && intervalValidation(other);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "WindowOffsetProcessor{" +
|
||||
internalToString() +
|
||||
", offset=" + offset +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,103 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.value;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
|
||||
import java.util.function.Function;
|
||||
|
||||
public abstract class WindowValueProcessorBase implements Processor
|
||||
{
|
||||
private final String inputColumn;
|
||||
private final String outputColumn;
|
||||
|
||||
public WindowValueProcessorBase(
|
||||
String inputColumn,
|
||||
String outputColumn
|
||||
)
|
||||
{
|
||||
this.inputColumn = inputColumn;
|
||||
this.outputColumn = outputColumn;
|
||||
}
|
||||
|
||||
@JsonProperty("inputColumn")
|
||||
public String getInputColumn()
|
||||
{
|
||||
return inputColumn;
|
||||
}
|
||||
|
||||
@JsonProperty("outputColumn")
|
||||
public String getOutputColumn()
|
||||
{
|
||||
return outputColumn;
|
||||
}
|
||||
|
||||
/**
|
||||
* This implements the common logic between the various value processors. It looks like it could be static, but if
|
||||
* it is static then the lambda becomes polymorphic. We keep it as a member method of the base class so taht the
|
||||
* JVM can inline it and specialize the lambda
|
||||
*
|
||||
* @param input incoming RowsAndColumns, as in Processor.process
|
||||
* @param fn function that converts the input column into the output column
|
||||
* @return RowsAndColumns, as in Processor.process
|
||||
*/
|
||||
public RowsAndColumns processInternal(RowsAndColumns input, Function<Column, Column> fn)
|
||||
{
|
||||
final AppendableRowsAndColumns retVal = RowsAndColumns.expectAppendable(input);
|
||||
|
||||
final Column column = input.findColumn(inputColumn);
|
||||
if (column == null) {
|
||||
throw new ISE("column[%s] doesn't exist, but window function FIRST wants it to", inputColumn);
|
||||
}
|
||||
|
||||
retVal.addColumn(outputColumn, fn.apply(column));
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(Processor otherProcessor)
|
||||
{
|
||||
return getClass() == otherProcessor.getClass()
|
||||
&& intervalValidation((WindowValueProcessorBase) otherProcessor);
|
||||
}
|
||||
|
||||
protected boolean intervalValidation(WindowValueProcessorBase other)
|
||||
{
|
||||
// Only input needs to be the same for the processors to produce equivalent results
|
||||
return inputColumn.equals(other.inputColumn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return getClass().getSimpleName() + "{" + internalToString() + '}';
|
||||
}
|
||||
|
||||
protected String internalToString()
|
||||
{
|
||||
return "inputColumn=" + inputColumn +
|
||||
", outputColumn='" + outputColumn + '\'';
|
||||
}
|
||||
}
|
|
@ -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.druid.query.rowsandcols;
|
||||
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
|
||||
/**
|
||||
* A RowsAndColumns that supports appending columns. This interface is particularly useful because even if there is
|
||||
* some composition of code that works with RowsAndColumns, we would like to add the columns to a singular base object
|
||||
* instead of build up a complex object graph.
|
||||
*/
|
||||
public interface AppendableRowsAndColumns extends RowsAndColumns
|
||||
{
|
||||
/**
|
||||
* Mutates the RowsAndColumns by appending the requested Column.
|
||||
*
|
||||
* @param name the name of the new column
|
||||
* @param column the Column object representing the new column
|
||||
*/
|
||||
void addColumn(String name, Column column);
|
||||
}
|
|
@ -0,0 +1,125 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols;
|
||||
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
import org.apache.druid.query.rowsandcols.column.ObjectColumnAccessorBase;
|
||||
import org.apache.druid.segment.RowAdapter;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
|
||||
public class ArrayListRowsAndColumns<RowType> implements RowsAndColumns
|
||||
{
|
||||
private final ArrayList<RowType> rows;
|
||||
private final RowAdapter<RowType> rowAdapter;
|
||||
private final RowSignature rowSignature;
|
||||
|
||||
public ArrayListRowsAndColumns(
|
||||
ArrayList<RowType> rows,
|
||||
RowAdapter<RowType> rowAdapter,
|
||||
RowSignature rowSignature
|
||||
)
|
||||
{
|
||||
this.rows = rows;
|
||||
this.rowAdapter = rowAdapter;
|
||||
this.rowSignature = rowSignature;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> getColumnNames()
|
||||
{
|
||||
return rowSignature.getColumnNames();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return rows.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Column findColumn(String name)
|
||||
{
|
||||
if (!rowSignature.contains(name)) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final Function<RowType, Object> adapterForValue = rowAdapter.columnFunction(name);
|
||||
final Optional<ColumnType> maybeColumnType = rowSignature.getColumnType(name);
|
||||
final ColumnType columnType = maybeColumnType.orElse(ColumnType.UNKNOWN_COMPLEX);
|
||||
final Comparator<Object> comparator = Comparator.nullsFirst(columnType.getStrategy());
|
||||
|
||||
return new Column()
|
||||
{
|
||||
@Override
|
||||
public ColumnAccessor toAccessor()
|
||||
{
|
||||
return new ObjectColumnAccessorBase()
|
||||
{
|
||||
@Override
|
||||
protected Object getVal(int cell)
|
||||
{
|
||||
return adapterForValue.apply(rows.get(cell));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Comparator<Object> getComparator()
|
||||
{
|
||||
return comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnType getType()
|
||||
{
|
||||
return columnType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return rows.size();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T as(Class<? extends T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public <T> T as(Class<T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,75 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
public class DefaultGroupPartitioner implements GroupPartitioner
|
||||
{
|
||||
private final RowsAndColumns rac;
|
||||
|
||||
public DefaultGroupPartitioner(
|
||||
RowsAndColumns rac
|
||||
)
|
||||
{
|
||||
this.rac = rac;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] computeGroupings(List<String> columns)
|
||||
{
|
||||
int[] retVal = new int[rac.numRows()];
|
||||
|
||||
for (String column : columns) {
|
||||
final Column theCol = rac.findColumn(column);
|
||||
if (theCol == null) {
|
||||
// The column doesn't exist. In this case, we assume it's always the same value: null. If it's always
|
||||
// the same, then it doesn't impact grouping at all and can be entirely skipped.
|
||||
continue;
|
||||
}
|
||||
final ColumnAccessor accessor = theCol.toAccessor();
|
||||
|
||||
int currGroup = 0;
|
||||
int prevGroupVal = 0;
|
||||
for (int i = 1; i < retVal.length; ++i) {
|
||||
if (retVal[i] == prevGroupVal) {
|
||||
int comparison = accessor.compareCells(i - 1, i);
|
||||
if (comparison == 0) {
|
||||
retVal[i] = currGroup;
|
||||
continue;
|
||||
} else if (comparison > 0) { // "greater than"
|
||||
throw new ISE("Pre-sorted data required, rows[%s] and [%s] were not in order", i - 1, i);
|
||||
} // the 3rd condition ("less than") means create a new group, so let it fall through
|
||||
}
|
||||
|
||||
// We have a new group, so walk things forward.
|
||||
prevGroupVal = retVal[i];
|
||||
retVal[i] = ++currGroup;
|
||||
}
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,271 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.aggregation.Aggregator;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
import org.apache.druid.segment.BaseSingleValueDimensionSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.serde.ComplexMetricSerde;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Function;
|
||||
|
||||
public class DefaultOnHeapAggregatable implements OnHeapAggregatable, OnHeapCumulativeAggregatable
|
||||
{
|
||||
private final RowsAndColumns rac;
|
||||
|
||||
public DefaultOnHeapAggregatable(
|
||||
RowsAndColumns rac
|
||||
)
|
||||
{
|
||||
this.rac = rac;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayList<Object> aggregateAll(
|
||||
List<AggregatorFactory> aggFactories
|
||||
)
|
||||
{
|
||||
Aggregator[] aggs = new Aggregator[aggFactories.size()];
|
||||
|
||||
AtomicInteger currRow = new AtomicInteger(0);
|
||||
int index = 0;
|
||||
for (AggregatorFactory aggFactory : aggFactories) {
|
||||
aggs[index++] = aggFactory.factorize(new ColumnAccessorBasedColumnSelectorFactory(currRow));
|
||||
}
|
||||
|
||||
int numRows = rac.numRows();
|
||||
int rowId = currRow.get();
|
||||
while (rowId < numRows) {
|
||||
for (Aggregator agg : aggs) {
|
||||
agg.aggregate();
|
||||
}
|
||||
rowId = currRow.incrementAndGet();
|
||||
}
|
||||
|
||||
ArrayList<Object> retVal = new ArrayList<>(aggs.length);
|
||||
for (Aggregator agg : aggs) {
|
||||
retVal.add(agg.get());
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayList<Object[]> aggregateCumulative(List<AggregatorFactory> aggFactories)
|
||||
{
|
||||
Aggregator[] aggs = new Aggregator[aggFactories.size()];
|
||||
ArrayList<Object[]> retVal = new ArrayList<>(aggFactories.size());
|
||||
|
||||
int numRows = rac.numRows();
|
||||
AtomicInteger currRow = new AtomicInteger(0);
|
||||
int index = 0;
|
||||
for (AggregatorFactory aggFactory : aggFactories) {
|
||||
aggs[index++] = aggFactory.factorize(new ColumnAccessorBasedColumnSelectorFactory(currRow));
|
||||
retVal.add(new Object[numRows]);
|
||||
}
|
||||
|
||||
int rowId = currRow.get();
|
||||
while (rowId < numRows) {
|
||||
for (int i = 0; i < aggs.length; ++i) {
|
||||
aggs[i].aggregate();
|
||||
retVal.get(i)[rowId] = aggs[i].get();
|
||||
}
|
||||
rowId = currRow.incrementAndGet();
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
private class ColumnAccessorBasedColumnSelectorFactory implements ColumnSelectorFactory
|
||||
{
|
||||
private final Map<String, ColumnAccessor> accessorCache = new HashMap<>();
|
||||
|
||||
private final AtomicInteger cellIdSupplier;
|
||||
|
||||
public ColumnAccessorBasedColumnSelectorFactory(AtomicInteger cellIdSupplier)
|
||||
{
|
||||
this.cellIdSupplier = cellIdSupplier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
|
||||
{
|
||||
return withColumnAccessor(dimensionSpec.getDimension(), columnAccessor -> {
|
||||
if (columnAccessor == null) {
|
||||
return DimensionSelector.constant(null);
|
||||
} else {
|
||||
return new BaseSingleValueDimensionSelector()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
protected String getValue()
|
||||
{
|
||||
return String.valueOf(columnAccessor.getObject(cellIdSupplier.get()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public ColumnValueSelector makeColumnValueSelector(@Nonnull String columnName)
|
||||
{
|
||||
return withColumnAccessor(columnName, columnAccessor -> {
|
||||
if (columnAccessor == null) {
|
||||
return DimensionSelector.constant(null);
|
||||
} else {
|
||||
return new ColumnValueSelector()
|
||||
{
|
||||
private final AtomicReference<Class> myClazz = new AtomicReference<>(null);
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object getObject()
|
||||
{
|
||||
return columnAccessor.getObject(cellIdSupplier.get());
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
Class retVal = myClazz.get();
|
||||
if (retVal == null) {
|
||||
retVal = findClazz();
|
||||
myClazz.set(retVal);
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
private Class findClazz()
|
||||
{
|
||||
final ColumnType type = columnAccessor.getType();
|
||||
switch (type.getType()) {
|
||||
case LONG:
|
||||
return long.class;
|
||||
case DOUBLE:
|
||||
return double.class;
|
||||
case FLOAT:
|
||||
return float.class;
|
||||
case STRING:
|
||||
return String.class;
|
||||
case ARRAY:
|
||||
return List.class;
|
||||
case COMPLEX:
|
||||
final ComplexMetricSerde serdeForType = ComplexMetrics.getSerdeForType(type.getComplexTypeName());
|
||||
if (serdeForType != null && serdeForType.getObjectStrategy() != null) {
|
||||
return serdeForType.getObjectStrategy().getClazz();
|
||||
}
|
||||
|
||||
for (int i = 0; i < columnAccessor.numRows(); ++i) {
|
||||
Object obj = columnAccessor.getObject(i);
|
||||
if (obj != null) {
|
||||
return obj.getClass();
|
||||
}
|
||||
}
|
||||
return Object.class;
|
||||
default:
|
||||
throw new ISE("Unknown type[%s]", type.getType());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
return columnAccessor.isNull(cellIdSupplier.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
return columnAccessor.getLong(cellIdSupplier.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
return columnAccessor.getFloat(cellIdSupplier.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble()
|
||||
{
|
||||
return columnAccessor.getDouble(cellIdSupplier.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ColumnCapabilities getColumnCapabilities(String column)
|
||||
{
|
||||
return withColumnAccessor(column, columnAccessor ->
|
||||
new ColumnCapabilitiesImpl()
|
||||
.setType(columnAccessor.getType())
|
||||
.setDictionaryEncoded(false)
|
||||
.setHasBitmapIndexes(false));
|
||||
}
|
||||
|
||||
private <T> T withColumnAccessor(String column, Function<ColumnAccessor, T> fn)
|
||||
{
|
||||
ColumnAccessor retVal = accessorCache.get(column);
|
||||
if (retVal == null) {
|
||||
Column racColumn = rac.findColumn(column);
|
||||
retVal = racColumn == null ? null : racColumn.toAccessor();
|
||||
accessorCache.put(column, retVal);
|
||||
}
|
||||
return fn.apply(retVal);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,93 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols;
|
||||
|
||||
import it.unimi.dsi.fastutil.ints.IntArrayList;
|
||||
import it.unimi.dsi.fastutil.ints.IntList;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.operator.LimitedRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class DefaultSortedGroupPartitioner implements SortedGroupPartitioner
|
||||
{
|
||||
private final RowsAndColumns rac;
|
||||
|
||||
public DefaultSortedGroupPartitioner(
|
||||
RowsAndColumns rac
|
||||
)
|
||||
{
|
||||
this.rac = rac;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] computeBoundaries(List<String> columns)
|
||||
{
|
||||
// Initialize to a grouping of everything
|
||||
IntList boundaries = new IntArrayList(new int[]{0, rac.numRows()});
|
||||
|
||||
for (String column : columns) {
|
||||
final Column theCol = rac.findColumn(column);
|
||||
if (theCol == null) {
|
||||
// The column doesn't exist. In this case, we assume it's always the same value: null. If it's always
|
||||
// the same, then it doesn't impact grouping at all and can be entirely skipped.
|
||||
continue;
|
||||
}
|
||||
final ColumnAccessor accessor = theCol.toAccessor();
|
||||
|
||||
IntList newBoundaries = new IntArrayList();
|
||||
newBoundaries.add(0);
|
||||
for (int i = 1; i < boundaries.size(); ++i) {
|
||||
int start = boundaries.getInt(i - 1);
|
||||
int end = boundaries.getInt(i);
|
||||
for (int j = start + 1; j < end; ++j) {
|
||||
int comparison = accessor.compareCells(j - 1, j);
|
||||
if (comparison < 0) {
|
||||
newBoundaries.add(j);
|
||||
} else if (comparison > 0) {
|
||||
throw new ISE("Pre-sorted data required, rows[%s] and [%s] were not in order", j - 1, j);
|
||||
}
|
||||
}
|
||||
newBoundaries.add(end);
|
||||
}
|
||||
boundaries = newBoundaries;
|
||||
}
|
||||
|
||||
return boundaries.toIntArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayList<RowsAndColumns> partitionOnBoundaries(List<String> partitionColumns)
|
||||
{
|
||||
final int[] boundaries = computeBoundaries(partitionColumns);
|
||||
ArrayList<RowsAndColumns> retVal = new ArrayList<>(boundaries.length - 1);
|
||||
|
||||
for (int i = 1; i < boundaries.length; ++i) {
|
||||
int start = boundaries[i - 1];
|
||||
int end = boundaries[i];
|
||||
retVal.add(new LimitedRowsAndColumns(rac, start, end));
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,43 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A semantic interface used to partition a data set based on a given set of dimensions.
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
public interface GroupPartitioner
|
||||
{
|
||||
/**
|
||||
* Computes the groupings of the underlying rows based on the columns passed in for grouping. The grouping is
|
||||
* returned as an int[], the length of the array will be equal to the number of rows of data and the values of
|
||||
* the elements of the array will be the same when the rows are part of the same group and different when the
|
||||
* rows are part of different groups. This is contrasted with the SortedGroupPartitioner in that, the
|
||||
* groupings returned are not necessarily contiguous. There is also no sort-order implied by the `int` values
|
||||
* assigned to each grouping.
|
||||
*
|
||||
* @param columns the columns to group with
|
||||
* @return the groupings, rows with the same int value are in the same group. There is no sort-order implied by the
|
||||
* int values.
|
||||
*/
|
||||
int[] computeGroupings(List<String> columns);
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols;
|
||||
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A semantic interface used to aggregate a list of AggregatorFactories across a given set of data
|
||||
* <p>
|
||||
* The aggregation specifically happens on-heap and should be used in places where it is known that the data
|
||||
* set can be worked with entirely on-heap.
|
||||
* <p>
|
||||
* Note, as we implement frame-handling for window aggregations, it is expected that this interface will undergo a
|
||||
* transformation. It might be deleted and replaced with something else, or might just see a change done in place.
|
||||
* Either way, there is no assumption of enforced compatibility with this interface at this point in time.
|
||||
*/
|
||||
public interface OnHeapAggregatable
|
||||
{
|
||||
/**
|
||||
* Aggregates the data using the {@code List<AggregatorFactory} objects.
|
||||
*
|
||||
* @param aggFactories definition of aggregations to be done
|
||||
* @return a list of objects, one per AggregatorFactory. That is, the length of the return list should be equal to
|
||||
* the length of the aggFactories list passed as an argument
|
||||
*/
|
||||
ArrayList<Object> aggregateAll(List<AggregatorFactory> aggFactories);
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols;
|
||||
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A semantic interface used to cumulatively aggregate a list of AggregatorFactories across a given set of data
|
||||
* <p>
|
||||
* The aggregation specifically happens on-heap and should be used in places where it is known that the data
|
||||
* set can be worked with entirely on-heap.
|
||||
* <p>
|
||||
* Note, as we implement frame-handling for window aggregations, it is expected that this interface will undergo a
|
||||
* transformation. It might be deleted and replaced with something else, or might just see a change done in place.
|
||||
* Either way, there is no assumption of enforced compatibility with this interface at this point in time.
|
||||
*/
|
||||
public interface OnHeapCumulativeAggregatable
|
||||
{
|
||||
/**
|
||||
* Cumulatively aggregates the data using the {@code List<AggregatorFactory} objects.
|
||||
*
|
||||
* @param aggFactories definition of aggregations to be done
|
||||
* @return a list of objects, one per AggregatorFactory. That is, the length of the return list should be equal to
|
||||
* the length of the aggFactories list passed as an argument, while the length of the internal {@code Object[]} will
|
||||
* be equivalent to the number of rows
|
||||
*/
|
||||
ArrayList<Object[]> aggregateCumulative(List<AggregatorFactory> aggFactories);
|
||||
}
|
|
@ -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.druid.query.rowsandcols;
|
||||
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.frame.AppendableMapOfColumns;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collection;
|
||||
|
||||
/**
|
||||
* An interface representing a chunk of RowsAndColumns. Essentially a RowsAndColumns is just a batch of rows
|
||||
* with columns.
|
||||
* <p>
|
||||
* This interface has very little prescriptively defined about what *must* be implemented. This is intentional
|
||||
* as there are lots of different possible representations of batch of rows each with their own unique positives
|
||||
* and negatives when it comes to processing. So, any explicit definition of what a RowsAndColumns is will actually,
|
||||
* by definition, end up as optimal for one specific configuration and sub-optimal for others. Instead of trying to
|
||||
* explicitly expand the interface to cover all the different possible ways that someone could want to interace
|
||||
* with a Rows and columns, we rely on semantic interfaces using the {@link RowsAndColumns#as} method instead.
|
||||
* <p>
|
||||
* That is, the expectation is that anything that works with a RowsAndColumns will tend to first ask the RowsAndColumns
|
||||
* object to become some other interface, for example, an {@link OnHeapAggregatable}. If a RowsAndColumns knows how
|
||||
* to do a good job as the requested interface, it can return its own concrete implementation of the interface and
|
||||
* run the necessary logic in its own optimized fashion. If the RowsAndColumns instance does not know how to implement
|
||||
* the semantic interface, it is expected that a default implementation of the interface can be instantiated on top of
|
||||
* the default column access mechanisms that the RowsAndColumns provides. Such default implementations should be
|
||||
* functionally correct, but are not believed to be optimal.
|
||||
* <p>
|
||||
* The "default column access mechanisms" here amount to using {@link #findColumn} to load a Column
|
||||
* and then using {@link Column#toAccessor} to access the individual cells of the column. There is also a
|
||||
* {@link Column#as} method which a default implementation might attempt to use to create a more optimal runtime.
|
||||
* <p>
|
||||
* It is intended that this interface can be used by Frames, Segments and even normal on-heap JVM data structures to
|
||||
* participate in query operations.
|
||||
*/
|
||||
public interface RowsAndColumns
|
||||
{
|
||||
@Nonnull
|
||||
static AppendableRowsAndColumns expectAppendable(RowsAndColumns input)
|
||||
{
|
||||
if (input instanceof AppendableRowsAndColumns) {
|
||||
return (AppendableRowsAndColumns) input;
|
||||
}
|
||||
|
||||
AppendableRowsAndColumns retVal = input.as(AppendableRowsAndColumns.class);
|
||||
if (retVal == null) {
|
||||
retVal = new AppendableMapOfColumns(input);
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
/**
|
||||
* The set of column names available from the RowsAndColumns
|
||||
*
|
||||
* @return The set of column names available from the RowsAndColumns
|
||||
*/
|
||||
@SuppressWarnings("unreachable")
|
||||
Collection<String> getColumnNames();
|
||||
|
||||
/**
|
||||
* The number of rows in the RowsAndColumns object
|
||||
*
|
||||
* @return the integer number of rows
|
||||
*/
|
||||
int numRows();
|
||||
|
||||
/**
|
||||
* Finds a column by name. null is returned if the column is not found. The RowsAndColumns object should not
|
||||
* attempt to default not-found columns to pretend as if they exist, instead the user of the RowsAndColumns object
|
||||
* should decide the correct semantic interpretation of a column that does not exist. It is expected that most
|
||||
* locations will choose to believe that the column does exist and is always null, but there are often optimizations
|
||||
* that can effect this same assumption without doing a lot of extra work if the calling code knows that it does not
|
||||
* exist.
|
||||
*
|
||||
* @param name the name of the column to find
|
||||
* @return the Column, if found. null if not found.
|
||||
*/
|
||||
Column findColumn(String name);
|
||||
|
||||
/**
|
||||
* Asks the RowsAndColumns to return itself as a concrete implementation of a specific interface. The interface
|
||||
* asked for will tend to be a semantically-meaningful interface. This method allows the calling code to interrogate
|
||||
* the RowsAndColumns object about whether it can offer a meaningful optimization of the semantic interface. If a
|
||||
* RowsAndColumns cannot do anything specifically optimal for the interface requested, it should return null instead
|
||||
* of trying to come up with its own default implementation.
|
||||
*
|
||||
* @param clazz A class object representing the interface that the calling code wants a concrete implementation of
|
||||
* @param <T> The interface that the calling code wants a concrete implementation of
|
||||
* @return A concrete implementation of the interface, or null if there is no meaningful optimization to be had
|
||||
* through a local implementation of the interface.
|
||||
*/
|
||||
@Nullable
|
||||
<T> T as(Class<T> clazz);
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A semantic interface used to partition a data set based on a given set of dimensions.
|
||||
* <p>
|
||||
* This specifically assumes that it is working with sorted data and, as such, the groups returned
|
||||
* should be contiguous and unique (that is, all rows for a given combination of values exist in only one grouping)
|
||||
*/
|
||||
public interface SortedGroupPartitioner
|
||||
{
|
||||
/**
|
||||
* Computes and returns a list of contiguous boundaries for independent groups. All rows in a specific grouping
|
||||
* should have the same values for the identified columns. Additionally, as this is assuming it is dealing with
|
||||
* sorted data, there should only be a single entry in the return value for a given set of values of the columns.
|
||||
*
|
||||
* @param columns the columns to partition on
|
||||
* @return an int[] representing the start (inclusive) and stop (exclusive) offsets of boundaries. Boundaries are
|
||||
* contiguous, so the stop of the previous boundary is the start of the subsequent one.
|
||||
*/
|
||||
int[] computeBoundaries(List<String> columns);
|
||||
|
||||
/**
|
||||
* Semantically equivalent to computeBoundaries, but returns a list of RowsAndColumns objects instead of just
|
||||
* boundary positions. This is useful as it allows the concrete implementation to return RowsAndColumns objects
|
||||
* that are aware of the internal representation of the data and thus can provide optimized implementations of
|
||||
* other semantic interfaces as the "child" RowsAndColumns are used
|
||||
*
|
||||
* @param partitionColumns the columns to partition on
|
||||
* @return a list of RowsAndColumns representing the data grouped by the partition columns.
|
||||
*/
|
||||
ArrayList<RowsAndColumns> partitionOnBoundaries(List<String> partitionColumns);
|
||||
}
|
|
@ -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.druid.query.rowsandcols.column;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
||||
/**
|
||||
* An interface representing a Column of data.
|
||||
*
|
||||
* This interface prescribes that a {@link ColumnAccessor} must be defined on the column, but also offers an
|
||||
* {@link #as} method to allow for optimized specific implementations of semantically meaningful logic.
|
||||
*
|
||||
* That is, the expectation is that some things work with Column objects might choose to first ask the Column
|
||||
* object to become some other interface. If the Column knows how to do a good job as the requested interface, it can
|
||||
* return its own concrete implementation of the interface and run the necessary logic in its own optimized fashion.
|
||||
* If the Column instance does not know how to implement the semantic interface, it is expected that the
|
||||
* {@link ColumnAccessor} will be leveraged to implement whatever logic is required.
|
||||
*/
|
||||
public interface Column
|
||||
{
|
||||
/**
|
||||
* Returns the column as a {@link ColumnAccessor}. Semantically, this would be equivalent to calling
|
||||
* {@Code Column.as(ColumnAccessor.class)}. However, being able to implement this interface is part of the explicit
|
||||
* contract of implementing this interface, so instead of relying on {@link #as} which allows for returning null,
|
||||
* we define a top-level method that should never return null.
|
||||
*
|
||||
* @return a {@link ColumnAccessor} representation of the column, this should never return null.
|
||||
*/
|
||||
@Nonnull
|
||||
ColumnAccessor toAccessor();
|
||||
|
||||
/**
|
||||
* Asks the Column to return itself as a concrete implementation of a specific interface. The interface
|
||||
* asked for will tend to be a semantically-meaningful interface. This method allows the calling code to interrogate
|
||||
* the Column object about whether it can offer a meaningful optimization of the semantic interface. If a
|
||||
* Column cannot do anything specifically optimal for the interface requested, it should return null instead
|
||||
* of trying to come up with its own default implementation.
|
||||
*
|
||||
* @param clazz A class object representing the interface that the calling code wants a concrete implementation of
|
||||
* @param <T> The interface that the calling code wants a concrete implementation of
|
||||
* @return A concrete implementation of the interface, or null if there is no meaningful optimization to be had
|
||||
* through a local implementation of the interface.
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
<T> T as(Class<? extends T> clazz);
|
||||
}
|
|
@ -0,0 +1,104 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols.column;
|
||||
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* Allows for accessing a column, provides methods to enable cell-by-cell access.
|
||||
*/
|
||||
public interface ColumnAccessor
|
||||
{
|
||||
/**
|
||||
* Get the type of the Column
|
||||
*
|
||||
* @return the type of the Column
|
||||
*/
|
||||
ColumnType getType();
|
||||
|
||||
/**
|
||||
* Get the number of cells
|
||||
*
|
||||
* @return the number of cells
|
||||
*/
|
||||
int numRows();
|
||||
|
||||
/**
|
||||
* Get whether the value of a cell is null
|
||||
*
|
||||
* @param rowNum the cell id, 0-indexed
|
||||
* @return true if the value is null
|
||||
*/
|
||||
boolean isNull(int rowNum);
|
||||
|
||||
/**
|
||||
* Get the {@link Object} representation of the cell.
|
||||
*
|
||||
* @param rowNum the cell id, 0-indexed
|
||||
* @return the {@link Object} representation of the cell. Returns {@code null} If {@link #isNull} is true.
|
||||
*/
|
||||
@Nullable
|
||||
Object getObject(int rowNum);
|
||||
|
||||
/**
|
||||
* Get the primitive {@code double} representation of the cell.
|
||||
*
|
||||
* @param rowNum the cell id, 0-indexed
|
||||
* @return the primitive {@code double} representation of the cell. Returns {@code 0D} If {@link #isNull} is true.
|
||||
*/
|
||||
double getDouble(int rowNum);
|
||||
|
||||
/**
|
||||
* Get the primitive {@code float} representation of the cell.
|
||||
*
|
||||
* @param rowNum the cell id, 0-indexed
|
||||
* @return the primitive {@code float} representation of the cell. Returns {@code 0F} If {@link #isNull} is true.
|
||||
*/
|
||||
float getFloat(int rowNum);
|
||||
|
||||
/**
|
||||
* Get the primitive {@code long} representation of the cell.
|
||||
*
|
||||
* @param rowNum the cell id, 0-indexed
|
||||
* @return the primitive {@code long} representation of the cell. Returns {@code 0L} If {@link #isNull} is true.
|
||||
*/
|
||||
long getLong(int rowNum);
|
||||
|
||||
/**
|
||||
* Get the primitive {@code int} representation of the cell.
|
||||
*
|
||||
* @param rowNum the cell id, 0-indexed
|
||||
* @return the primitive {@code int} representation of the cell. Returns {@code 0} If {@link #isNull} is true.
|
||||
*/
|
||||
int getInt(int rowNum);
|
||||
|
||||
/**
|
||||
* Compares two cells using a comparison that follows the same semantics as {@link java.util.Comparator#compare}
|
||||
* <p>
|
||||
* This is not comparing the cell Ids, but the values referred to by the cell ids.
|
||||
*
|
||||
* @param lhsRowNum the cell id of the left-hand-side of the comparison
|
||||
* @param rhsRowNum the cell id of the right-hand-side of the comparison
|
||||
* @return the result of the comparison of the two cells
|
||||
*/
|
||||
int compareCells(int lhsRowNum, int rhsRowNum);
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols.column;
|
||||
|
||||
public class ColumnAccessorBasedColumn implements Column
|
||||
{
|
||||
private final ColumnAccessor base;
|
||||
|
||||
public ColumnAccessorBasedColumn(
|
||||
ColumnAccessor base
|
||||
)
|
||||
{
|
||||
this.base = base;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnAccessor toAccessor()
|
||||
{
|
||||
return base;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T as(Class<? extends T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,103 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols.column;
|
||||
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
public class ConstantObjectColumn implements Column
|
||||
{
|
||||
private final Object obj;
|
||||
private final int numCells;
|
||||
private final ColumnType type;
|
||||
|
||||
public ConstantObjectColumn(Object obj, int numCells, ColumnType type)
|
||||
{
|
||||
this.obj = obj;
|
||||
this.numCells = numCells;
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnAccessor toAccessor()
|
||||
{
|
||||
return new ColumnAccessor()
|
||||
{
|
||||
@Override
|
||||
public ColumnType getType()
|
||||
{
|
||||
return type;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return numCells;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull(int rowNum)
|
||||
{
|
||||
return obj == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getObject(int rowNum)
|
||||
{
|
||||
return obj;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble(int rowNum)
|
||||
{
|
||||
return ((Number) obj).doubleValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(int rowNum)
|
||||
{
|
||||
return ((Number) obj).floatValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(int rowNum)
|
||||
{
|
||||
return ((Number) obj).longValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(int rowNum)
|
||||
{
|
||||
return ((Number) obj).intValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareCells(int lhsRowNum, int rhsRowNum)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T as(Class<? extends T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,101 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols.column;
|
||||
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
public class DoubleArrayColumn implements Column
|
||||
{
|
||||
private final double[] vals;
|
||||
|
||||
public DoubleArrayColumn(
|
||||
double[] vals
|
||||
)
|
||||
{
|
||||
this.vals = vals;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnAccessor toAccessor()
|
||||
{
|
||||
return new ColumnAccessor()
|
||||
{
|
||||
@Override
|
||||
public ColumnType getType()
|
||||
{
|
||||
return ColumnType.DOUBLE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return vals.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull(int rowNum)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getObject(int rowNum)
|
||||
{
|
||||
return vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble(int rowNum)
|
||||
{
|
||||
return vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(int rowNum)
|
||||
{
|
||||
return (float) vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(int rowNum)
|
||||
{
|
||||
return (long) vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(int rowNum)
|
||||
{
|
||||
return (int) vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareCells(int lhsRowNum, int rhsRowNum)
|
||||
{
|
||||
return Double.compare(lhsRowNum, rhsRowNum);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T as(Class<? extends T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,101 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols.column;
|
||||
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
public class IntArrayColumn implements Column
|
||||
{
|
||||
private final int[] vals;
|
||||
|
||||
public IntArrayColumn(
|
||||
int[] vals
|
||||
)
|
||||
{
|
||||
this.vals = vals;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnAccessor toAccessor()
|
||||
{
|
||||
return new ColumnAccessor()
|
||||
{
|
||||
@Override
|
||||
public ColumnType getType()
|
||||
{
|
||||
return ColumnType.LONG;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return vals.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull(int rowNum)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getObject(int rowNum)
|
||||
{
|
||||
return vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble(int rowNum)
|
||||
{
|
||||
return vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(int rowNum)
|
||||
{
|
||||
return vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(int rowNum)
|
||||
{
|
||||
return vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(int rowNum)
|
||||
{
|
||||
return vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareCells(int lhsRowNum, int rhsRowNum)
|
||||
{
|
||||
return Integer.compare(vals[lhsRowNum], vals[rhsRowNum]);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T as(Class<? extends T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols.column;
|
||||
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
public class NullColumnAccessor implements ColumnAccessor
|
||||
{
|
||||
private final ColumnType type;
|
||||
private final int size;
|
||||
|
||||
public NullColumnAccessor(int size)
|
||||
{
|
||||
this(ColumnType.UNKNOWN_COMPLEX, size);
|
||||
}
|
||||
|
||||
public NullColumnAccessor(ColumnType type, int size)
|
||||
{
|
||||
this.type = type;
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnType getType()
|
||||
{
|
||||
return type;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull(int rowNum)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object getObject(int rowNum)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble(int rowNum)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(int rowNum)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(int rowNum)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(int rowNum)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareCells(int lhsRowNum, int rhsRowNum)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
|
@ -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.druid.query.rowsandcols.column;
|
||||
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
import java.util.Comparator;
|
||||
|
||||
public class ObjectArrayColumn implements Column
|
||||
{
|
||||
private final Object[] objects;
|
||||
private final ColumnType resultType;
|
||||
private final Comparator<Object> comparator;
|
||||
|
||||
public ObjectArrayColumn(Object[] objects, ColumnType resultType)
|
||||
{
|
||||
this(objects, resultType, Comparator.nullsFirst(resultType.getStrategy()));
|
||||
}
|
||||
|
||||
public ObjectArrayColumn(Object[] objects, ColumnType resultType, Comparator<Object> comparator)
|
||||
{
|
||||
this.objects = objects;
|
||||
this.resultType = resultType;
|
||||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnAccessor toAccessor()
|
||||
{
|
||||
return new ObjectColumnAccessorBase()
|
||||
{
|
||||
@Override
|
||||
protected Object getVal(int cell)
|
||||
{
|
||||
return objects[cell];
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Comparator<Object> getComparator()
|
||||
{
|
||||
return comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnType getType()
|
||||
{
|
||||
return resultType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return objects.length;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T as(Class<? extends T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,121 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols.column;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Comparator;
|
||||
|
||||
public abstract class ObjectColumnAccessorBase implements ColumnAccessor
|
||||
{
|
||||
@Override
|
||||
public boolean isNull(int rowNum)
|
||||
{
|
||||
return getVal(rowNum) == null;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object getObject(int rowNum)
|
||||
{
|
||||
return getVal(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble(int rowNum)
|
||||
{
|
||||
final Object val = getVal(rowNum);
|
||||
if (val instanceof Number) {
|
||||
return ((Number) val).doubleValue();
|
||||
} else if (val instanceof String) {
|
||||
try {
|
||||
return Double.parseDouble((String) val);
|
||||
}
|
||||
catch (NumberFormatException e) {
|
||||
return 0d;
|
||||
}
|
||||
} else {
|
||||
return 0d;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(int rowNum)
|
||||
{
|
||||
final Object val = getVal(rowNum);
|
||||
if (val instanceof Number) {
|
||||
return ((Number) val).floatValue();
|
||||
} else if (val instanceof String) {
|
||||
try {
|
||||
return Float.parseFloat((String) val);
|
||||
}
|
||||
catch (NumberFormatException e) {
|
||||
return 0f;
|
||||
}
|
||||
} else {
|
||||
return 0f;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(int rowNum)
|
||||
{
|
||||
final Object val = getVal(rowNum);
|
||||
if (val instanceof Number) {
|
||||
return ((Number) val).longValue();
|
||||
} else if (val instanceof String) {
|
||||
try {
|
||||
return Long.parseLong((String) val);
|
||||
}
|
||||
catch (NumberFormatException e) {
|
||||
return 0L;
|
||||
}
|
||||
} else {
|
||||
return 0L;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(int rowNum)
|
||||
{
|
||||
final Object val = getVal(rowNum);
|
||||
if (val instanceof Number) {
|
||||
return ((Number) val).intValue();
|
||||
} else if (val instanceof String) {
|
||||
try {
|
||||
return Integer.parseInt((String) val);
|
||||
}
|
||||
catch (NumberFormatException e) {
|
||||
return 0;
|
||||
}
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareCells(int lhsRowNum, int rhsRowNum)
|
||||
{
|
||||
return getComparator().compare(getVal(lhsRowNum), getVal(rhsRowNum));
|
||||
}
|
||||
|
||||
protected abstract Object getVal(int cell);
|
||||
|
||||
protected abstract Comparator<Object> getComparator();
|
||||
}
|
|
@ -0,0 +1,93 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols.frame;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.Set;
|
||||
|
||||
public class AppendableMapOfColumns implements AppendableRowsAndColumns
|
||||
{
|
||||
private final RowsAndColumns base;
|
||||
private final LinkedHashMap<String, Column> appendedColumns;
|
||||
private Set<String> colNames = null;
|
||||
|
||||
public AppendableMapOfColumns(
|
||||
RowsAndColumns base
|
||||
)
|
||||
{
|
||||
this.base = base;
|
||||
this.appendedColumns = new LinkedHashMap<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addColumn(String name, Column column)
|
||||
{
|
||||
final Column prevValue = appendedColumns.put(name, column);
|
||||
if (prevValue != null) {
|
||||
throw new ISE("Tried to override column[%s]!? Was[%s], now[%s]", name, prevValue, column);
|
||||
}
|
||||
if (colNames != null) {
|
||||
colNames.add(name);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> getColumnNames()
|
||||
{
|
||||
if (colNames == null) {
|
||||
this.colNames = new LinkedHashSet<>(base.getColumnNames());
|
||||
this.colNames.addAll(appendedColumns.keySet());
|
||||
}
|
||||
return colNames;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return base.numRows();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Column findColumn(String name)
|
||||
{
|
||||
Column retVal = base.findColumn(name);
|
||||
if (retVal == null) {
|
||||
retVal = appendedColumns.get(name);
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public <T> T as(Class<T> clazz)
|
||||
{
|
||||
if (AppendableRowsAndColumns.class.equals(clazz)) {
|
||||
return (T) this;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,109 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols.frame;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class MapOfColumnsRowsAndColumns implements RowsAndColumns
|
||||
{
|
||||
public static MapOfColumnsRowsAndColumns of(String name, Column col)
|
||||
{
|
||||
return fromMap(ImmutableMap.of(name, col));
|
||||
}
|
||||
|
||||
public static MapOfColumnsRowsAndColumns of(String name, Column col, String name2, Column col2)
|
||||
{
|
||||
return fromMap(ImmutableMap.of(name, col, name2, col2));
|
||||
}
|
||||
|
||||
public static MapOfColumnsRowsAndColumns fromMap(Map<String, Column> map)
|
||||
{
|
||||
if (map == null || map.isEmpty()) {
|
||||
throw new ISE("map[%s] cannot be null or empty.", map);
|
||||
}
|
||||
|
||||
final Iterator<Map.Entry<String, Column>> iter = map.entrySet().iterator();
|
||||
Map.Entry<String, Column> entry = iter.next();
|
||||
int numCells = entry.getValue().toAccessor().numRows();
|
||||
if (iter.hasNext()) {
|
||||
entry = iter.next();
|
||||
final int newCells = entry.getValue().toAccessor().numRows();
|
||||
if (numCells != newCells) {
|
||||
throw new ISE(
|
||||
"Mismatched numCells, expectedNumCells[%s], actual[%s] from col[%s].",
|
||||
numCells,
|
||||
newCells,
|
||||
entry.getKey()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
return new MapOfColumnsRowsAndColumns(map, map.values().iterator().next().toAccessor().numRows());
|
||||
}
|
||||
|
||||
private final Map<String, Column> mapOfColumns;
|
||||
private final int numRows;
|
||||
|
||||
public MapOfColumnsRowsAndColumns(
|
||||
Map<String, Column> mapOfColumns,
|
||||
int numRows
|
||||
)
|
||||
{
|
||||
this.mapOfColumns = mapOfColumns;
|
||||
this.numRows = numRows;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getColumnNames()
|
||||
{
|
||||
return mapOfColumns.keySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return numRows;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Column findColumn(String name)
|
||||
{
|
||||
return mapOfColumns.get(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public <T> T as(Class<T> clazz)
|
||||
{
|
||||
if (AppendableRowsAndColumns.class.equals(clazz)) {
|
||||
return (T) new AppendableMapOfColumns(this);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
|
@ -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.druid.segment;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.query.rowsandcols.ArrayListRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
* A {@link Segment} that is based on a stream of objects.
|
||||
*/
|
||||
public class ArrayListSegment<RowType> implements Segment
|
||||
{
|
||||
private final SegmentId segmentId;
|
||||
private final ArrayList<RowType> rows;
|
||||
private final RowAdapter<RowType> rowAdapter;
|
||||
private final RowSignature rowSignature;
|
||||
|
||||
/**
|
||||
* Create a list-based segment.
|
||||
* <p>
|
||||
* The provided List must be in time-order according to the provided {@link RowAdapter#timestampFunction()}.
|
||||
* The cursor returned by {@link RowBasedStorageAdapter#makeCursors} makes no attempt to verify this, and callers
|
||||
* will expect it.
|
||||
* <p>
|
||||
* The provided "rowSignature" will be used for reporting available columns and their capabilities to users of
|
||||
* {@link #asStorageAdapter()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's
|
||||
* storage adapter will allow creation of selectors on any field, using the {@link RowAdapter#columnFunction} for that
|
||||
* field, even if it doesn't appear in "rowSignature".
|
||||
*
|
||||
* @param segmentId segment identifier; will be returned by {@link #getId()}
|
||||
* @param rows objects that comprise this segment. Must be re-iterable if support for {@link Cursor#reset()}
|
||||
* is required. Otherwise, does not need to be re-iterable.
|
||||
* @param rowAdapter adapter used for reading these objects
|
||||
* @param rowSignature signature of the columns in these objects
|
||||
*/
|
||||
public ArrayListSegment(
|
||||
final SegmentId segmentId,
|
||||
final ArrayList<RowType> rows,
|
||||
final RowAdapter<RowType> rowAdapter,
|
||||
final RowSignature rowSignature
|
||||
)
|
||||
{
|
||||
this.segmentId = Preconditions.checkNotNull(segmentId, "segmentId");
|
||||
this.rows = rows;
|
||||
this.rowAdapter = rowAdapter;
|
||||
this.rowSignature = rowSignature;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nonnull
|
||||
public SegmentId getId()
|
||||
{
|
||||
return segmentId;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nonnull
|
||||
public Interval getDataInterval()
|
||||
{
|
||||
return Intervals.ETERNITY;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public QueryableIndex asQueryableIndex()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nonnull
|
||||
public StorageAdapter asStorageAdapter()
|
||||
{
|
||||
return new RowBasedStorageAdapter<>(Sequences.simple(rows), rowAdapter, rowSignature);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public <T> T as(Class<T> clazz)
|
||||
{
|
||||
if (RowsAndColumns.class.equals(clazz)) {
|
||||
return (T) asRowsAndColumns();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
// Do nothing.
|
||||
}
|
||||
|
||||
private RowsAndColumns asRowsAndColumns()
|
||||
{
|
||||
return new ArrayListRowsAndColumns(rows, rowAdapter, rowSignature);
|
||||
}
|
||||
|
||||
}
|
|
@ -50,7 +50,6 @@ public abstract class BaseSingleValueDimensionSelector implements DimensionSelec
|
|||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
assert id == 0;
|
||||
return getValue();
|
||||
}
|
||||
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.druid.guice.annotations.PublicApi;
|
|||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
|
||||
|
@ -57,9 +58,9 @@ public interface Segment extends Closeable
|
|||
* @param <T> desired interface
|
||||
* @return instance of clazz, or null if the interface is not supported by this segment
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
@SuppressWarnings({"unused", "unchecked"})
|
||||
@Nullable
|
||||
default <T> T as(Class<T> clazz)
|
||||
default <T> T as(@Nonnull Class<T> clazz)
|
||||
{
|
||||
if (clazz.equals(QueryableIndex.class)) {
|
||||
return (T) asQueryableIndex();
|
||||
|
|
|
@ -82,9 +82,9 @@ public class ObjectStrategyComplexTypeStrategy<T> implements TypeStrategy<T>
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compare(T o1, T o2)
|
||||
public int compare(Object o1, Object o2)
|
||||
{
|
||||
return objectStrategy.compare(o1, o2);
|
||||
return objectStrategy.compare((T) o1, (T) o2);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -407,9 +407,9 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compare(Integer o1, Integer o2)
|
||||
public int compare(Object o1, Object o2)
|
||||
{
|
||||
return Integer.compare(o1, o2);
|
||||
return Integer.compare(((Number) o1).intValue(), ((Number) o2).intValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,73 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Iterators;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
public class InlineScanOperator implements Operator
|
||||
{
|
||||
public static InlineScanOperator make(RowsAndColumns item)
|
||||
{
|
||||
return new InlineScanOperator(Iterators.singletonIterator(item));
|
||||
}
|
||||
|
||||
public static InlineScanOperator make(List<RowsAndColumns> items)
|
||||
{
|
||||
return new InlineScanOperator(items.iterator());
|
||||
}
|
||||
|
||||
private Iterator<RowsAndColumns> iter;
|
||||
|
||||
public InlineScanOperator(
|
||||
Iterator<RowsAndColumns> iter
|
||||
)
|
||||
{
|
||||
Preconditions.checkNotNull(iter);
|
||||
this.iter = iter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open()
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowsAndColumns next()
|
||||
{
|
||||
return iter.next();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
return iter.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(boolean cascade)
|
||||
{
|
||||
iter = null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,105 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
public class NaivePartitioningOperatorTest
|
||||
{
|
||||
@Test
|
||||
public void testDefaultImplementation()
|
||||
{
|
||||
RowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(
|
||||
ImmutableMap.of(
|
||||
"sorted", new IntArrayColumn(new int[]{0, 0, 0, 1, 1, 2, 4, 4, 4}),
|
||||
"unsorted", new IntArrayColumn(new int[]{3, 54, 21, 1, 5, 54, 2, 3, 92})
|
||||
)
|
||||
);
|
||||
|
||||
NaivePartitioningOperator op = new NaivePartitioningOperator(
|
||||
ImmutableList.of("sorted"),
|
||||
InlineScanOperator.make(rac)
|
||||
);
|
||||
|
||||
op.open();
|
||||
|
||||
List<RowsAndColumnsHelper> expectations = Arrays.asList(
|
||||
new RowsAndColumnsHelper()
|
||||
.expectColumn("sorted", new int[]{0, 0, 0})
|
||||
.expectColumn("unsorted", new int[]{3, 54, 21}),
|
||||
new RowsAndColumnsHelper()
|
||||
.expectColumn("sorted", new int[]{1, 1})
|
||||
.expectColumn("unsorted", new int[]{1, 5}),
|
||||
new RowsAndColumnsHelper()
|
||||
.expectColumn("sorted", new int[]{2})
|
||||
.expectColumn("unsorted", new int[]{54}),
|
||||
new RowsAndColumnsHelper()
|
||||
.expectColumn("sorted", new int[]{4, 4, 4})
|
||||
.expectColumn("unsorted", new int[]{2, 3, 92})
|
||||
);
|
||||
|
||||
for (RowsAndColumnsHelper expectation : expectations) {
|
||||
Assert.assertTrue(op.hasNext());
|
||||
expectation.validate(op.next());
|
||||
}
|
||||
Assert.assertFalse(op.hasNext());
|
||||
|
||||
op.close(true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailUnsorted()
|
||||
{
|
||||
RowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(
|
||||
ImmutableMap.of(
|
||||
"sorted", new IntArrayColumn(new int[]{0, 0, 0, 1, 1, 2, 4, 4, 4}),
|
||||
"unsorted", new IntArrayColumn(new int[]{3, 54, 21, 1, 5, 54, 2, 3, 92})
|
||||
)
|
||||
);
|
||||
|
||||
NaivePartitioningOperator op = new NaivePartitioningOperator(
|
||||
ImmutableList.of("unsorted"),
|
||||
InlineScanOperator.make(rac)
|
||||
);
|
||||
|
||||
op.open();
|
||||
|
||||
boolean exceptionThrown = false;
|
||||
try {
|
||||
op.next();
|
||||
}
|
||||
catch (ISE ex) {
|
||||
Assert.assertEquals("Pre-sorted data required, rows[1] and [2] were not in order", ex.getMessage());
|
||||
exceptionThrown = true;
|
||||
}
|
||||
Assert.assertTrue(exceptionThrown);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,56 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import org.apache.druid.java.util.common.guava.Yielder;
|
||||
import org.apache.druid.java.util.common.guava.YieldingAccumulator;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class OperatorSequenceTest
|
||||
{
|
||||
@Test
|
||||
public void testSanity()
|
||||
{
|
||||
OperatorSequence seq = new OperatorSequence(
|
||||
() -> InlineScanOperator.make(MapOfColumnsRowsAndColumns.of("hi", new IntArrayColumn(new int[]{1})))
|
||||
);
|
||||
|
||||
Assert.assertEquals(1, seq.accumulate(0, (accumulated, in) -> accumulated + 1).intValue());
|
||||
|
||||
Yielder<Integer> yielder = seq.toYielder(0, new YieldingAccumulator<Integer, RowsAndColumns>()
|
||||
{
|
||||
@Override
|
||||
public Integer accumulate(Integer accumulated, RowsAndColumns in)
|
||||
{
|
||||
yield();
|
||||
return accumulated + 1;
|
||||
}
|
||||
});
|
||||
Assert.assertFalse(yielder.isDone());
|
||||
Assert.assertEquals(1, yielder.get().intValue());
|
||||
|
||||
yielder = yielder.next(0);
|
||||
Assert.assertTrue(yielder.isDone());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,56 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class SequenceOperatorTest
|
||||
{
|
||||
@Test
|
||||
public void testSanity()
|
||||
{
|
||||
SequenceOperator op = new SequenceOperator(Sequences.simple(Arrays.asList(
|
||||
MapOfColumnsRowsAndColumns.of("hi", new IntArrayColumn(new int[]{1})),
|
||||
MapOfColumnsRowsAndColumns.of("hi", new IntArrayColumn(new int[]{1}))
|
||||
)));
|
||||
|
||||
op.open();
|
||||
|
||||
RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("hi", new int[]{1})
|
||||
.allColumnsRegistered();
|
||||
|
||||
expectations.validate(op.next());
|
||||
Assert.assertTrue(op.hasNext());
|
||||
|
||||
expectations.validate(op.next());
|
||||
Assert.assertFalse(op.hasNext());
|
||||
|
||||
op.close(true);
|
||||
op.close(false);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,124 @@
|
|||
/*
|
||||
* 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.druid.query.operator;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.QueryContext;
|
||||
import org.apache.druid.query.TableDataSource;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Tests the WindowOperatorQuery, it would actually be a lot better to run this through some tests that actually
|
||||
* validate the operation of queries, but all of the efforts to build out test scaffolding and framework have gone
|
||||
* into building things out for SQL query operations. As such, all of the tests that validating the actual native
|
||||
* functionality actually run from the `druid-sql` module instead of this module. It would be best to de-couple
|
||||
* these and have all of the native, query processing tests happen directly here in processing and have the SQL
|
||||
* tests only concern themselves with how they plan SQL into Native, but that's a bit big of a nugget to bite off
|
||||
* at this point in time, so instead we continue the building of technical debt by making this "test" run lines
|
||||
* of code without actually testing much meaningful behavior.
|
||||
* <p>
|
||||
* For now, view CalciteWindowQueryTest for actual tests that validate behavior.
|
||||
*/
|
||||
public class WindowOperatorQueryTest
|
||||
{
|
||||
WindowOperatorQuery query;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
query = new WindowOperatorQuery(
|
||||
InlineDataSource.fromIterable(new ArrayList<>(), RowSignature.empty()),
|
||||
ImmutableMap.of("sally", "sue"),
|
||||
RowSignature.empty(),
|
||||
new ArrayList<>()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void getOperators()
|
||||
{
|
||||
Assert.assertTrue(query.getOperators().isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void getRowSignature()
|
||||
{
|
||||
Assert.assertEquals(0, query.getRowSignature().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void hasFilters()
|
||||
{
|
||||
Assert.assertFalse(query.hasFilters());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void getFilter()
|
||||
{
|
||||
Assert.assertNull(query.getFilter());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void getType()
|
||||
{
|
||||
Assert.assertEquals("windowOperator", query.getType());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void withOverriddenContext()
|
||||
{
|
||||
Assert.assertEquals("sue", query.context().get("sally"));
|
||||
final QueryContext context = query.withOverriddenContext(ImmutableMap.of("sally", "soo")).context();
|
||||
Assert.assertEquals("soo", context.get("sally"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void withDataSource()
|
||||
{
|
||||
final Set<String> tableNames = query.getDataSource().getTableNames();
|
||||
Assert.assertEquals(0, tableNames.size());
|
||||
|
||||
boolean exceptionThrown = false;
|
||||
try {
|
||||
query.withDataSource(new TableDataSource("bob"));
|
||||
}
|
||||
catch (IAE e) {
|
||||
// should fail trying to set a TableDataSource as TableDataSource is not currently allowed.
|
||||
exceptionThrown = true;
|
||||
}
|
||||
Assert.assertTrue(exceptionThrown);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEquals()
|
||||
{
|
||||
Assert.assertEquals(query, query);
|
||||
Assert.assertEquals(query, query.withDataSource(query.getDataSource()));
|
||||
Assert.assertNotEquals(query, query.toString());
|
||||
}
|
||||
}
|
|
@ -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.druid.query.operator;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class WindowProcessorOperatorTest
|
||||
{
|
||||
@Test
|
||||
public void testJustRunsTheProcessor()
|
||||
{
|
||||
RowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(
|
||||
ImmutableMap.of(
|
||||
"colA", new IntArrayColumn(new int[]{1, 2, 3}),
|
||||
"colB", new IntArrayColumn(new int[]{3, 2, 1})
|
||||
)
|
||||
);
|
||||
|
||||
WindowProcessorOperator op = new WindowProcessorOperator(
|
||||
new Processor()
|
||||
{
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns incomingPartition)
|
||||
{
|
||||
return incomingPartition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(Processor otherProcessor)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
},
|
||||
InlineScanOperator.make(rac)
|
||||
);
|
||||
|
||||
op.open();
|
||||
Assert.assertTrue(op.hasNext());
|
||||
Assert.assertSame(rac, op.next());
|
||||
Assert.assertFalse(op.hasNext());
|
||||
op.close(true);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window;
|
||||
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class ComposingProcessorTest
|
||||
{
|
||||
@Test
|
||||
public void testSanity()
|
||||
{
|
||||
final ProcessorForTesting firstProcessor = new ProcessorForTesting();
|
||||
final ProcessorForTesting secondProcessor = new ProcessorForTesting();
|
||||
|
||||
ComposingProcessor proc = new ComposingProcessor(firstProcessor, secondProcessor);
|
||||
|
||||
proc.process(null);
|
||||
Assert.assertEquals(1, firstProcessor.processCounter);
|
||||
Assert.assertEquals(1, secondProcessor.processCounter);
|
||||
|
||||
proc.process(null);
|
||||
Assert.assertEquals(2, firstProcessor.processCounter);
|
||||
Assert.assertEquals(2, secondProcessor.processCounter);
|
||||
|
||||
Assert.assertTrue(proc.validateEquivalent(proc));
|
||||
Assert.assertEquals(1, firstProcessor.validateCounter);
|
||||
Assert.assertEquals(1, secondProcessor.validateCounter);
|
||||
|
||||
firstProcessor.validationResult = false;
|
||||
Assert.assertFalse(proc.validateEquivalent(proc));
|
||||
Assert.assertEquals(2, firstProcessor.validateCounter);
|
||||
Assert.assertEquals(1, secondProcessor.validateCounter);
|
||||
}
|
||||
|
||||
private static class ProcessorForTesting implements Processor
|
||||
{
|
||||
private int processCounter = 0;
|
||||
private int validateCounter = 0;
|
||||
private boolean validationResult = true;
|
||||
|
||||
@Override
|
||||
public RowsAndColumns process(RowsAndColumns incomingPartition)
|
||||
{
|
||||
++processCounter;
|
||||
return incomingPartition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validateEquivalent(Processor otherProcessor)
|
||||
{
|
||||
++validateCounter;
|
||||
return validationResult;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,269 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.junit.Assert;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class RowsAndColumnsHelper
|
||||
{
|
||||
public static void assertEquals(RowsAndColumns rac, String name, int[] expectedResults)
|
||||
{
|
||||
final Column column = rac.findColumn(name);
|
||||
Assert.assertNotNull(column);
|
||||
final ColumnAccessor accessor = column.toAccessor();
|
||||
Assert.assertEquals(expectedResults.length, accessor.numRows());
|
||||
for (int i = 0; i < expectedResults.length; ++i) {
|
||||
Assert.assertEquals(StringUtils.format("%s[%s]", name, i), expectedResults[i], accessor.getInt(i));
|
||||
}
|
||||
}
|
||||
|
||||
public static void assertEquals(RowsAndColumns rac, String name, long[] expectedResults)
|
||||
{
|
||||
final Column column = rac.findColumn(name);
|
||||
Assert.assertNotNull(column);
|
||||
final ColumnAccessor accessor = column.toAccessor();
|
||||
Assert.assertEquals(expectedResults.length, accessor.numRows());
|
||||
for (int i = 0; i < expectedResults.length; ++i) {
|
||||
Assert.assertEquals(StringUtils.format("%s[%s]", name, i), expectedResults[i], accessor.getLong(i));
|
||||
}
|
||||
}
|
||||
|
||||
public static void assertEquals(RowsAndColumns rac, String name, double[] expectedResults)
|
||||
{
|
||||
final Column column = rac.findColumn(name);
|
||||
Assert.assertNotNull(column);
|
||||
final ColumnAccessor accessor = column.toAccessor();
|
||||
Assert.assertEquals(expectedResults.length, accessor.numRows());
|
||||
for (int i = 0; i < expectedResults.length; ++i) {
|
||||
Assert.assertEquals(StringUtils.format("%s[%s]", name, i), expectedResults[i], accessor.getDouble(i), 0.0d);
|
||||
}
|
||||
}
|
||||
|
||||
private final Map<String, ColumnHelper> helpers = new LinkedHashMap<>();
|
||||
private Set<String> fullColumnSet;
|
||||
|
||||
public RowsAndColumnsHelper()
|
||||
{
|
||||
}
|
||||
|
||||
public RowsAndColumnsHelper expectColumn(String col, int[] expectedVals)
|
||||
{
|
||||
final ColumnHelper helper = columnHelper(col, expectedVals.length, ColumnType.LONG);
|
||||
helper.setExpectation(expectedVals);
|
||||
return this;
|
||||
}
|
||||
|
||||
public RowsAndColumnsHelper expectColumn(String col, long[] expectedVals)
|
||||
{
|
||||
final ColumnHelper helper = columnHelper(col, expectedVals.length, ColumnType.LONG);
|
||||
helper.setExpectation(expectedVals);
|
||||
return this;
|
||||
}
|
||||
|
||||
public RowsAndColumnsHelper expectColumn(String col, double[] expectedVals)
|
||||
{
|
||||
final ColumnHelper helper = columnHelper(col, expectedVals.length, ColumnType.DOUBLE);
|
||||
helper.setExpectation(expectedVals);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnHelper columnHelper(String column, int expectedSize, ColumnType expectedType)
|
||||
{
|
||||
ColumnHelper retVal = helpers.get(column);
|
||||
if (retVal == null) {
|
||||
retVal = new ColumnHelper(expectedSize, expectedType);
|
||||
helpers.put(column, retVal);
|
||||
return retVal;
|
||||
} else {
|
||||
throw new ISE(
|
||||
"column[%s] expectations already defined, size[%s], type[%s]",
|
||||
column,
|
||||
retVal.expectedVals.length,
|
||||
retVal.expectedType
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
public RowsAndColumnsHelper expectFullColumns(Set<String> fullColumnSet)
|
||||
{
|
||||
this.fullColumnSet = fullColumnSet;
|
||||
return this;
|
||||
}
|
||||
|
||||
public RowsAndColumnsHelper allColumnsRegistered()
|
||||
{
|
||||
this.fullColumnSet = ImmutableSet.copyOf(helpers.keySet());
|
||||
return this;
|
||||
}
|
||||
|
||||
public void validate(RowsAndColumns rac)
|
||||
{
|
||||
validate("", rac);
|
||||
}
|
||||
|
||||
public void validate(String name, RowsAndColumns rac)
|
||||
{
|
||||
if (fullColumnSet != null) {
|
||||
final Collection<String> columnNames = rac.getColumnNames();
|
||||
Assert.assertEquals(name, fullColumnSet.size(), columnNames.size());
|
||||
Assert.assertTrue(name, fullColumnSet.containsAll(columnNames));
|
||||
}
|
||||
|
||||
for (Map.Entry<String, ColumnHelper> entry : helpers.entrySet()) {
|
||||
entry.getValue().validate(StringUtils.format("%s.%s", name, entry.getKey()), rac.findColumn(entry.getKey()));
|
||||
}
|
||||
}
|
||||
|
||||
public static class ColumnHelper
|
||||
{
|
||||
private final ColumnType expectedType;
|
||||
private final Object[] expectedVals;
|
||||
private final boolean[] expectedNulls;
|
||||
|
||||
public ColumnHelper(int expectedSize, ColumnType expectedType)
|
||||
{
|
||||
this.expectedType = expectedType;
|
||||
this.expectedVals = new Object[expectedSize];
|
||||
this.expectedNulls = new boolean[expectedVals.length];
|
||||
}
|
||||
|
||||
public ColumnHelper setExpectation(int[] expectedVals)
|
||||
{
|
||||
for (int i = 0; i < expectedVals.length; i++) {
|
||||
this.expectedVals[i] = expectedVals[i];
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnHelper setExpectation(long[] expectedVals)
|
||||
{
|
||||
for (int i = 0; i < expectedVals.length; i++) {
|
||||
this.expectedVals[i] = expectedVals[i];
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnHelper setExpectation(double[] expectedVals)
|
||||
{
|
||||
for (int i = 0; i < expectedVals.length; i++) {
|
||||
this.expectedVals[i] = expectedVals[i];
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnHelper setExpectation(float[] expectedVals)
|
||||
{
|
||||
for (int i = 0; i < expectedVals.length; i++) {
|
||||
this.expectedVals[i] = expectedVals[i];
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnHelper setExpectation(Object[] expectedVals)
|
||||
{
|
||||
System.arraycopy(expectedVals, 0, this.expectedVals, 0, expectedVals.length);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnHelper setNulls(int[] nullIndexes)
|
||||
{
|
||||
for (int nullIndex : nullIndexes) {
|
||||
this.expectedNulls[nullIndex] = true;
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public void validate(String msgBase, Column col)
|
||||
{
|
||||
final ColumnAccessor accessor = col.toAccessor();
|
||||
|
||||
Assert.assertEquals(msgBase, expectedType, accessor.getType());
|
||||
Assert.assertEquals(msgBase, expectedVals.length, accessor.numRows());
|
||||
for (int i = 0; i < accessor.numRows(); ++i) {
|
||||
final String msg = StringUtils.format("%s[%s]", msgBase, i);
|
||||
Object expectedVal = expectedVals[i];
|
||||
if (expectedVal == null) {
|
||||
Assert.assertTrue(msg, expectedNulls[i]);
|
||||
Assert.assertTrue(msg, accessor.isNull(i));
|
||||
Assert.assertNull(msg, accessor.getObject(i));
|
||||
}
|
||||
if (expectedVal instanceof Float) {
|
||||
if (expectedNulls[i]) {
|
||||
Assert.assertTrue(msg, accessor.isNull(i));
|
||||
Assert.assertEquals(msg, 0.0f, accessor.getFloat(i), 0.0);
|
||||
} else {
|
||||
Assert.assertFalse(msg, accessor.isNull(i));
|
||||
Assert.assertEquals(msg, (Float) expectedVal, accessor.getFloat(i), 0.0);
|
||||
}
|
||||
} else if (expectedVal instanceof Double) {
|
||||
if (expectedNulls[i]) {
|
||||
Assert.assertTrue(msg, accessor.isNull(i));
|
||||
Assert.assertEquals(msg, 0.0d, accessor.getDouble(i), 0.0);
|
||||
} else {
|
||||
Assert.assertFalse(msg, accessor.isNull(i));
|
||||
Assert.assertEquals(msg, (Double) expectedVal, accessor.getDouble(i), 0.0);
|
||||
}
|
||||
} else if (expectedVal instanceof Integer) {
|
||||
if (expectedNulls[i]) {
|
||||
Assert.assertTrue(msg, accessor.isNull(i));
|
||||
Assert.assertEquals(msg, 0, accessor.getInt(i));
|
||||
} else {
|
||||
Assert.assertFalse(msg, accessor.isNull(i));
|
||||
Assert.assertEquals(msg, ((Integer) expectedVal).intValue(), accessor.getInt(i));
|
||||
}
|
||||
} else if (expectedVal instanceof Long) {
|
||||
if (expectedNulls[i]) {
|
||||
Assert.assertTrue(msg, accessor.isNull(i));
|
||||
Assert.assertEquals(msg, 0, accessor.getLong(i));
|
||||
} else {
|
||||
Assert.assertFalse(msg, accessor.isNull(i));
|
||||
Assert.assertEquals(msg, ((Long) expectedVal).longValue(), accessor.getLong(i));
|
||||
}
|
||||
} else {
|
||||
if (expectedNulls[i]) {
|
||||
Assert.assertTrue(msg, accessor.isNull(i));
|
||||
Assert.assertNull(msg, accessor.getObject(i));
|
||||
// asserting null on the expected value is here for consistency in the tests. If it fails, it's most
|
||||
// likely indicative of something wrong with the test setup than the actual logic, we keep it for
|
||||
// sanity's sake to things consistent.
|
||||
Assert.assertNull(msg, expectedVals[i]);
|
||||
} else {
|
||||
final Object obj = accessor.getObject(i);
|
||||
Assert.assertFalse(msg, accessor.isNull(i));
|
||||
Assert.assertNotNull(msg, obj);
|
||||
Assert.assertEquals(msg, expectedVals[i], obj);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,115 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window;
|
||||
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.query.operator.window.ranking.WindowRowNumberProcessor;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class WindowAggregateProcessorTest
|
||||
{
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAggregation()
|
||||
{
|
||||
Map<String, Column> map = new LinkedHashMap<>();
|
||||
map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("doubleCol", new DoubleArrayColumn(new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("objectCol", new ObjectArrayColumn(
|
||||
new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"},
|
||||
ColumnType.STRING
|
||||
)
|
||||
);
|
||||
|
||||
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
|
||||
|
||||
WindowAggregateProcessor processor = new WindowAggregateProcessor(
|
||||
Arrays.asList(
|
||||
new LongSumAggregatorFactory("sumFromLong", "intCol"),
|
||||
new LongSumAggregatorFactory("sumFromDouble", "doubleCol"),
|
||||
new DoubleMaxAggregatorFactory("maxFromInt", "intCol"),
|
||||
new DoubleMaxAggregatorFactory("maxFromDouble", "doubleCol")
|
||||
),
|
||||
Arrays.asList(
|
||||
new LongMaxAggregatorFactory("cummMax", "intCol"),
|
||||
new DoubleSumAggregatorFactory("cummSum", "doubleCol")
|
||||
)
|
||||
);
|
||||
|
||||
RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("doubleCol", new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("sumFromLong", new long[]{45, 45, 45, 45, 45, 45, 45, 45, 45, 45})
|
||||
.expectColumn("sumFromDouble", new long[]{45, 45, 45, 45, 45, 45, 45, 45, 45, 45})
|
||||
.expectColumn("maxFromInt", new double[]{9, 9, 9, 9, 9, 9, 9, 9, 9, 9})
|
||||
.expectColumn("maxFromDouble", new double[]{9, 9, 9, 9, 9, 9, 9, 9, 9, 9})
|
||||
.expectColumn("cummMax", new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("cummSum", new double[]{0, 1, 3, 6, 10, 15, 21, 28, 36, 45});
|
||||
|
||||
final RowsAndColumns results = processor.process(rac);
|
||||
expectations.validate(results);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidateEquality()
|
||||
{
|
||||
WindowAggregateProcessor processor = new WindowAggregateProcessor(
|
||||
Arrays.asList(
|
||||
new LongSumAggregatorFactory("sumFromLong", "intCol"),
|
||||
new LongSumAggregatorFactory("sumFromDouble", "doubleCol"),
|
||||
new DoubleMaxAggregatorFactory("maxFromInt", "intCol"),
|
||||
new DoubleMaxAggregatorFactory("maxFromDouble", "doubleCol")
|
||||
),
|
||||
Arrays.asList(
|
||||
new LongMaxAggregatorFactory("cummMax", "intCol"),
|
||||
new DoubleSumAggregatorFactory("cummSum", "doubleCol")
|
||||
)
|
||||
);
|
||||
|
||||
Assert.assertTrue(processor.validateEquivalent(processor));
|
||||
Assert.assertFalse(processor.validateEquivalent(new WindowRowNumberProcessor("bob")));
|
||||
Assert.assertFalse(processor.validateEquivalent(new WindowAggregateProcessor(processor.getAggregations(), null)));
|
||||
Assert.assertFalse(processor.validateEquivalent(
|
||||
new WindowAggregateProcessor(new ArrayList<>(), processor.getCumulativeAggregations())
|
||||
));
|
||||
Assert.assertFalse(processor.validateEquivalent(new WindowAggregateProcessor(new ArrayList<>(), null)));
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.ranking;
|
||||
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class WindowCumeDistProcessorTest
|
||||
{
|
||||
@Test
|
||||
public void testCumeDistProcessing()
|
||||
{
|
||||
Map<String, Column> map = new LinkedHashMap<>();
|
||||
map.put("vals", new IntArrayColumn(new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290}));
|
||||
|
||||
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
|
||||
|
||||
Processor processor = new WindowCumeDistProcessor(Collections.singletonList("vals"), "CumeDist");
|
||||
|
||||
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("vals", new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290})
|
||||
.expectColumn("CumeDist", new double[]{0.1, 0.3, 0.3, 0.4, 0.5, 0.6, 0.8, 0.8, 1.0, 1.0});
|
||||
|
||||
final RowsAndColumns results = processor.process(rac);
|
||||
expectations.validate(results);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.ranking;
|
||||
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class WindowDenseRankProcessorTest
|
||||
{
|
||||
@Test
|
||||
public void testDenseRankProcessing()
|
||||
{
|
||||
Map<String, Column> map = new LinkedHashMap<>();
|
||||
map.put("vals", new IntArrayColumn(new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290}));
|
||||
|
||||
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
|
||||
|
||||
Processor processor = new WindowDenseRankProcessor(Collections.singletonList("vals"), "DenseRank");
|
||||
|
||||
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("vals", new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290})
|
||||
.expectColumn("DenseRank", new int[]{1, 2, 2, 3, 4, 5, 6, 6, 7, 7});
|
||||
|
||||
final RowsAndColumns results = processor.process(rac);
|
||||
expectations.validate(results);
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,84 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.ranking;
|
||||
|
||||
import org.apache.druid.query.operator.window.ComposingProcessor;
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class WindowPercentileProcessorTest
|
||||
{
|
||||
@Test
|
||||
public void testPercentileProcessing()
|
||||
{
|
||||
Map<String, Column> map = new LinkedHashMap<>();
|
||||
map.put("intCol", new IntArrayColumn(new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("doubleCol", new DoubleArrayColumn(new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("objectCol", new ObjectArrayColumn(
|
||||
new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"},
|
||||
ColumnType.STRING
|
||||
));
|
||||
|
||||
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
|
||||
|
||||
Processor processor = new ComposingProcessor(
|
||||
new WindowPercentileProcessor("1", 1),
|
||||
new WindowPercentileProcessor("2", 2),
|
||||
new WindowPercentileProcessor("3", 3),
|
||||
new WindowPercentileProcessor("4", 4),
|
||||
new WindowPercentileProcessor("5", 5),
|
||||
new WindowPercentileProcessor("6", 6),
|
||||
new WindowPercentileProcessor("7", 7),
|
||||
new WindowPercentileProcessor("8", 8),
|
||||
new WindowPercentileProcessor("9", 9),
|
||||
new WindowPercentileProcessor("10", 10),
|
||||
new WindowPercentileProcessor("10292", 10292)
|
||||
);
|
||||
|
||||
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("intCol", new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("doubleCol", new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("1", new int[]{1, 1, 1, 1, 1, 1, 1, 1, 1, 1})
|
||||
.expectColumn("2", new int[]{1, 1, 1, 1, 1, 2, 2, 2, 2, 2})
|
||||
.expectColumn("3", new int[]{1, 1, 1, 1, 2, 2, 2, 3, 3, 3})
|
||||
.expectColumn("4", new int[]{1, 1, 1, 2, 2, 2, 3, 3, 4, 4})
|
||||
.expectColumn("5", new int[]{1, 1, 2, 2, 3, 3, 4, 4, 5, 5})
|
||||
.expectColumn("6", new int[]{1, 1, 2, 2, 3, 3, 4, 4, 5, 6})
|
||||
.expectColumn("7", new int[]{1, 1, 2, 2, 3, 3, 4, 5, 6, 7})
|
||||
.expectColumn("8", new int[]{1, 1, 2, 2, 3, 4, 5, 6, 7, 8})
|
||||
.expectColumn("9", new int[]{1, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("10", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
.expectColumn("10292", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10});
|
||||
|
||||
final RowsAndColumns results = processor.process(rac);
|
||||
expectations.validate(results);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,84 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.ranking;
|
||||
|
||||
import org.apache.druid.query.operator.window.ComposingProcessor;
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class WindowRankProcessorTest
|
||||
{
|
||||
@Test
|
||||
public void testRankProcessing()
|
||||
{
|
||||
Map<String, Column> map = new LinkedHashMap<>();
|
||||
map.put("vals", new IntArrayColumn(new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290}));
|
||||
|
||||
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
|
||||
|
||||
Processor processor = new ComposingProcessor(
|
||||
new WindowRankProcessor(Collections.singletonList("vals"), "rank", false),
|
||||
new WindowRankProcessor(Collections.singletonList("vals"), "rankAsPercent", true)
|
||||
);
|
||||
|
||||
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("vals", new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290})
|
||||
.expectColumn("rank", new int[]{1, 2, 2, 4, 5, 6, 7, 7, 9, 9})
|
||||
.expectColumn(
|
||||
"rankAsPercent",
|
||||
new double[]{0.0, 1 / 9d, 1 / 9d, 3 / 9d, 4 / 9d, 5 / 9d, 6 / 9d, 6 / 9d, 8 / 9d, 8 / 9d}
|
||||
);
|
||||
|
||||
final RowsAndColumns results = processor.process(rac);
|
||||
expectations.validate(results);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRankSingle()
|
||||
{
|
||||
Map<String, Column> map = new LinkedHashMap<>();
|
||||
map.put("vals", new IntArrayColumn(new int[]{7}));
|
||||
|
||||
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
|
||||
|
||||
Processor processor = new ComposingProcessor(
|
||||
new WindowRankProcessor(Collections.singletonList("vals"), "rank", false),
|
||||
new WindowRankProcessor(Collections.singletonList("vals"), "rankAsPercent", true)
|
||||
);
|
||||
|
||||
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("vals", new int[]{7})
|
||||
.expectColumn("rank", new int[]{1})
|
||||
.expectColumn("rankAsPercent", new double[]{0.0});
|
||||
|
||||
final RowsAndColumns results = processor.process(rac);
|
||||
expectations.validate(results);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.ranking;
|
||||
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class WindowRowNumberProcessorTest
|
||||
{
|
||||
@Test
|
||||
public void testRowNumberProcessing()
|
||||
{
|
||||
Map<String, Column> map = new LinkedHashMap<>();
|
||||
map.put("intCol", new IntArrayColumn(new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("doubleCol", new DoubleArrayColumn(new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("objectCol", new ObjectArrayColumn(
|
||||
new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"},
|
||||
ColumnType.STRING
|
||||
));
|
||||
|
||||
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
|
||||
|
||||
Processor processor = new WindowRowNumberProcessor("rowRow");
|
||||
|
||||
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("intCol", new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("doubleCol", new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("rowRow", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10});
|
||||
|
||||
final RowsAndColumns results = processor.process(rac);
|
||||
expectations.validate(results);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.value;
|
||||
|
||||
import org.apache.druid.query.operator.window.ComposingProcessor;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class WindowFirstProcessorTest
|
||||
{
|
||||
@Test
|
||||
public void testFirstProcessing()
|
||||
{
|
||||
Map<String, Column> map = new LinkedHashMap<>();
|
||||
map.put("intCol", new IntArrayColumn(new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("doubleCol", new DoubleArrayColumn(new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("objectCol", new ObjectArrayColumn(
|
||||
new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"},
|
||||
ColumnType.STRING
|
||||
));
|
||||
map.put("nullFirstCol", new ObjectArrayColumn(
|
||||
new String[]{null, "b", "c", "d", "e", "f", "g", "h", "i", "j"},
|
||||
ColumnType.STRING
|
||||
));
|
||||
|
||||
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
|
||||
|
||||
ComposingProcessor processor = new ComposingProcessor(
|
||||
new WindowFirstProcessor("intCol", "FirstIntCol"),
|
||||
new WindowFirstProcessor("doubleCol", "FirstDoubleCol"),
|
||||
new WindowFirstProcessor("objectCol", "FirstObjectCol"),
|
||||
new WindowFirstProcessor("nullFirstCol", "NullFirstCol")
|
||||
);
|
||||
|
||||
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("intCol", new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("doubleCol", new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("FirstIntCol", new int[]{88, 88, 88, 88, 88, 88, 88, 88, 88, 88})
|
||||
.expectColumn(
|
||||
"FirstDoubleCol",
|
||||
new double[]{0.4728, 0.4728, 0.4728, 0.4728, 0.4728, 0.4728, 0.4728, 0.4728, 0.4728, 0.4728}
|
||||
);
|
||||
|
||||
expectations.columnHelper("FirstObjectCol", 10, ColumnType.STRING)
|
||||
.setExpectation(new String[]{"a", "a", "a", "a", "a", "a", "a", "a", "a", "a"});
|
||||
|
||||
expectations.columnHelper("NullFirstCol", 10, ColumnType.STRING)
|
||||
.setNulls(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9});
|
||||
|
||||
final RowsAndColumns results = processor.process(rac);
|
||||
expectations.validate(results);
|
||||
}
|
||||
}
|
|
@ -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.druid.query.operator.window.value;
|
||||
|
||||
import org.apache.druid.query.operator.window.ComposingProcessor;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class WindowLagProcessorTest
|
||||
{
|
||||
@Test
|
||||
public void testLagProcessing()
|
||||
{
|
||||
Map<String, Column> map = new LinkedHashMap<>();
|
||||
map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("doubleCol", new DoubleArrayColumn(new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("objectCol", new ObjectArrayColumn(
|
||||
new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"},
|
||||
ColumnType.STRING
|
||||
)
|
||||
);
|
||||
|
||||
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
|
||||
|
||||
ComposingProcessor processor = new ComposingProcessor(
|
||||
new WindowOffsetProcessor("intCol", "laggardIntCol", -2),
|
||||
new WindowOffsetProcessor("doubleCol", "laggardDoubleCol", -4),
|
||||
new WindowOffsetProcessor("objectCol", "laggardObjectCol", -1)
|
||||
);
|
||||
|
||||
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("doubleCol", new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9});
|
||||
|
||||
expectations.columnHelper("laggardIntCol", 10, ColumnType.LONG)
|
||||
.setExpectation(new int[]{0, 0, 0, 1, 2, 3, 4, 5, 6, 7})
|
||||
.setNulls(new int[]{0, 1});
|
||||
|
||||
expectations.columnHelper("laggardDoubleCol", 10, ColumnType.DOUBLE)
|
||||
.setExpectation(new double[]{0, 0, 0, 0, 0, 1, 2, 3, 4, 5})
|
||||
.setNulls(new int[]{0, 1, 2, 3});
|
||||
|
||||
expectations.columnHelper("laggardObjectCol", 10, ColumnType.STRING)
|
||||
.setExpectation(new String[]{null, "a", "b", "c", "d", "e", "f", "g", "h", "i"})
|
||||
.setNulls(new int[]{0});
|
||||
|
||||
final RowsAndColumns results = processor.process(rac);
|
||||
expectations.validate(results);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.value;
|
||||
|
||||
import org.apache.druid.query.operator.window.ComposingProcessor;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class WindowLastProcessorTest
|
||||
{
|
||||
@Test
|
||||
public void testLastProcessing()
|
||||
{
|
||||
Map<String, Column> map = new LinkedHashMap<>();
|
||||
map.put("intCol", new IntArrayColumn(new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("doubleCol", new DoubleArrayColumn(new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9.84}));
|
||||
map.put("objectCol", new ObjectArrayColumn(
|
||||
new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"},
|
||||
ColumnType.STRING
|
||||
));
|
||||
map.put("nullLastCol", new ObjectArrayColumn(
|
||||
new String[]{null, "b", "c", "d", "e", "f", "g", "h", "i", null},
|
||||
ColumnType.STRING
|
||||
));
|
||||
|
||||
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
|
||||
|
||||
ComposingProcessor processor = new ComposingProcessor(
|
||||
new WindowLastProcessor("intCol", "LastIntCol"),
|
||||
new WindowLastProcessor("doubleCol", "LastDoubleCol"),
|
||||
new WindowLastProcessor("objectCol", "LastObjectCol"),
|
||||
new WindowLastProcessor("nullLastCol", "NullLastCol")
|
||||
);
|
||||
|
||||
|
||||
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("intCol", new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("doubleCol", new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9.84})
|
||||
.expectColumn("LastIntCol", new int[]{9, 9, 9, 9, 9, 9, 9, 9, 9, 9})
|
||||
.expectColumn("LastDoubleCol", new double[]{9.84, 9.84, 9.84, 9.84, 9.84, 9.84, 9.84, 9.84, 9.84, 9.84});
|
||||
|
||||
expectations.columnHelper("LastObjectCol", 10, ColumnType.STRING)
|
||||
.setExpectation(new String[]{"j", "j", "j", "j", "j", "j", "j", "j", "j", "j"});
|
||||
|
||||
expectations.columnHelper("NullLastCol", 10, ColumnType.STRING)
|
||||
.setNulls(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9});
|
||||
|
||||
final RowsAndColumns results = processor.process(rac);
|
||||
expectations.validate(results);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* 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.druid.query.operator.window.value;
|
||||
|
||||
import org.apache.druid.query.operator.window.ComposingProcessor;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class WindowLeadProcessorTest
|
||||
{
|
||||
@Test
|
||||
public void testLeadProcessing()
|
||||
{
|
||||
Map<String, Column> map = new LinkedHashMap<>();
|
||||
map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("doubleCol", new DoubleArrayColumn(new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
map.put("objectCol", new ObjectArrayColumn(
|
||||
new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"},
|
||||
ColumnType.STRING
|
||||
)
|
||||
);
|
||||
|
||||
MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map);
|
||||
|
||||
ComposingProcessor processor = new ComposingProcessor(
|
||||
new WindowOffsetProcessor("intCol", "LeadingIntCol", 2),
|
||||
new WindowOffsetProcessor("doubleCol", "LeadingDoubleCol", 4),
|
||||
new WindowOffsetProcessor("objectCol", "LeadingObjectCol", 1)
|
||||
);
|
||||
|
||||
final RowsAndColumns results = processor.process(rac);
|
||||
|
||||
final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper()
|
||||
.expectColumn("intCol", new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
.expectColumn("doubleCol", new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9});
|
||||
|
||||
expectations.columnHelper("LeadingIntCol", 10, ColumnType.LONG)
|
||||
.setExpectation(new int[]{2, 3, 4, 5, 6, 7, 8, 9, 0, 0})
|
||||
.setNulls(new int[]{8, 9});
|
||||
|
||||
expectations.columnHelper("LeadingDoubleCol", 10, ColumnType.DOUBLE)
|
||||
.setExpectation(new double[]{4, 5, 6, 7, 8, 9, 0, 0, 0, 0})
|
||||
.setNulls(new int[]{6, 7, 8, 9});
|
||||
|
||||
expectations.columnHelper("LeadingObjectCol", 10, ColumnType.STRING)
|
||||
.setExpectation(new String[]{"b", "c", "d", "e", "f", "g", "h", "i", "j", null})
|
||||
.setNulls(new int[]{9});
|
||||
}
|
||||
}
|
|
@ -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.druid.query.rowsandcols;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
||||
public class ArrayListRowsAndColumnsTest extends RowsAndColumnsTestBase<ArrayListRowsAndColumns<Object[]>>
|
||||
{
|
||||
|
||||
@Override
|
||||
public ArrayListRowsAndColumns<Object[]> makeRowsAndColumns(MapOfColumnsRowsAndColumns input)
|
||||
{
|
||||
ArrayList<Object[]> rows = new ArrayList<>(input.numRows());
|
||||
|
||||
ArrayList<String> cols = new ArrayList<>(input.getColumnNames());
|
||||
final RowSignature.Builder sigBob = RowSignature.builder();
|
||||
|
||||
for (int i = 0; i < input.numRows(); ++i) {
|
||||
rows.add(new Object[cols.size()]);
|
||||
}
|
||||
|
||||
for (int colIndex = 0; colIndex < cols.size(); ++colIndex) {
|
||||
String col = cols.get(colIndex);
|
||||
final ColumnAccessor column = input.findColumn(col).toAccessor();
|
||||
sigBob.add(col, column.getType());
|
||||
|
||||
for (int i = 0; i < column.numRows(); ++i) {
|
||||
rows.get(i)[colIndex] = column.getObject(i);
|
||||
}
|
||||
}
|
||||
|
||||
return new ArrayListRowsAndColumns<>(
|
||||
rows,
|
||||
columnName -> {
|
||||
final int i = cols.indexOf(columnName);
|
||||
if (i < 0) {
|
||||
throw new ISE("Couldn't find column[%s]!? i[%s]", columnName, i);
|
||||
}
|
||||
return objects -> objects[i];
|
||||
},
|
||||
sigBob.build()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Collections;
|
||||
|
||||
public class MapOfColumnsRowsAndColumnsTest extends RowsAndColumnsTestBase<MapOfColumnsRowsAndColumns>
|
||||
{
|
||||
@Override
|
||||
public MapOfColumnsRowsAndColumns makeRowsAndColumns(MapOfColumnsRowsAndColumns input)
|
||||
{
|
||||
return input;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMakeWithEmptyAndNull()
|
||||
{
|
||||
boolean exceptionThrown = false;
|
||||
try {
|
||||
MapOfColumnsRowsAndColumns.fromMap(null);
|
||||
}
|
||||
catch (ISE ex) {
|
||||
Assert.assertEquals("map[null] cannot be null or empty.", ex.getMessage());
|
||||
exceptionThrown = true;
|
||||
}
|
||||
Assert.assertTrue(exceptionThrown);
|
||||
|
||||
exceptionThrown = false;
|
||||
try {
|
||||
MapOfColumnsRowsAndColumns.fromMap(Collections.emptyMap());
|
||||
}
|
||||
catch (ISE ex) {
|
||||
Assert.assertEquals("map[{}] cannot be null or empty.", ex.getMessage());
|
||||
exceptionThrown = true;
|
||||
}
|
||||
Assert.assertTrue(exceptionThrown);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExceptionOnMismatchedCells()
|
||||
{
|
||||
boolean exceptionThrown = false;
|
||||
try {
|
||||
MapOfColumnsRowsAndColumns.of(
|
||||
"1", new IntArrayColumn(new int[]{0}),
|
||||
"2", new IntArrayColumn(new int[]{0, 1})
|
||||
);
|
||||
}
|
||||
catch (ISE ex) {
|
||||
Assert.assertEquals("Mismatched numCells, expectedNumCells[1], actual[2] from col[2].", ex.getMessage());
|
||||
exceptionThrown = true;
|
||||
}
|
||||
Assert.assertTrue(exceptionThrown);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,188 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.frame.AppendableMapOfColumns;
|
||||
import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* This base class is intended to serve as a common set of tests to validate specific RowsAndColumns implementations.
|
||||
* <p>
|
||||
* Different RowsAndColumns implementations will implement different of the semantic interfaces, this base class should
|
||||
* test all of the possible semantic interfaces that can be implemented. By doing it this way, we can ensure that
|
||||
* new RowsAndColumns implementations meet all of the corners cases and other issues that have been previously found.
|
||||
* <p>
|
||||
* It is expected that this base class is going to grow quite large. As it gets extra large, we could perhaps look
|
||||
* into whether one of the JUnit test runners could allow us to further sub-divide the test functionality into
|
||||
* semantic-interface-specific tests. The ultimate goal, however, should be that a new RowsAndColumns implementation
|
||||
* can very simply take advantage of all of the tests by implementing the abstract
|
||||
* {@link #makeRowsAndColumns(MapOfColumnsRowsAndColumns)} method and be done.
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public abstract class RowsAndColumnsTestBase<T extends RowsAndColumns>
|
||||
{
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
}
|
||||
|
||||
public abstract T makeRowsAndColumns(MapOfColumnsRowsAndColumns input);
|
||||
|
||||
@Test
|
||||
public void testDefaultSortedGroupPartitioner()
|
||||
{
|
||||
T rac = makeRowsAndColumns(MapOfColumnsRowsAndColumns.fromMap(
|
||||
ImmutableMap.of(
|
||||
"sorted", new IntArrayColumn(new int[]{0, 0, 0, 1, 1, 2, 4, 4, 4}),
|
||||
"unsorted", new IntArrayColumn(new int[]{3, 54, 21, 1, 5, 54, 2, 3, 92})
|
||||
)
|
||||
));
|
||||
|
||||
validateSortedGroupPartitioner("default", new DefaultSortedGroupPartitioner(rac));
|
||||
|
||||
SortedGroupPartitioner specialized = rac.as(SortedGroupPartitioner.class);
|
||||
if (specialized != null) {
|
||||
validateSortedGroupPartitioner("specialized", specialized);
|
||||
}
|
||||
}
|
||||
|
||||
private void validateSortedGroupPartitioner(String name, SortedGroupPartitioner parter)
|
||||
{
|
||||
|
||||
int[] expectedBounds = new int[]{0, 3, 5, 6, 9};
|
||||
|
||||
List<RowsAndColumnsHelper> expectations = Arrays.asList(
|
||||
new RowsAndColumnsHelper()
|
||||
.expectColumn("sorted", new int[]{0, 0, 0})
|
||||
.expectColumn("unsorted", new int[]{3, 54, 21})
|
||||
.allColumnsRegistered(),
|
||||
new RowsAndColumnsHelper()
|
||||
.expectColumn("sorted", new int[]{1, 1})
|
||||
.expectColumn("unsorted", new int[]{1, 5})
|
||||
.allColumnsRegistered(),
|
||||
new RowsAndColumnsHelper()
|
||||
.expectColumn("sorted", new int[]{2})
|
||||
.expectColumn("unsorted", new int[]{54})
|
||||
.allColumnsRegistered(),
|
||||
new RowsAndColumnsHelper()
|
||||
.expectColumn("sorted", new int[]{4, 4, 4})
|
||||
.expectColumn("unsorted", new int[]{2, 3, 92})
|
||||
.allColumnsRegistered()
|
||||
);
|
||||
|
||||
final List<String> partCols = Collections.singletonList("sorted");
|
||||
Assert.assertArrayEquals(name, expectedBounds, parter.computeBoundaries(partCols));
|
||||
|
||||
final Iterator<RowsAndColumns> partedChunks = parter.partitionOnBoundaries(partCols).iterator();
|
||||
for (RowsAndColumnsHelper expectation : expectations) {
|
||||
Assert.assertTrue(name, partedChunks.hasNext());
|
||||
expectation.validate(name, partedChunks.next());
|
||||
}
|
||||
Assert.assertFalse(name, partedChunks.hasNext());
|
||||
|
||||
boolean exceptionThrown = false;
|
||||
try {
|
||||
parter.partitionOnBoundaries(Collections.singletonList("unsorted"));
|
||||
}
|
||||
catch (ISE ex) {
|
||||
Assert.assertEquals("Pre-sorted data required, rows[1] and [2] were not in order", ex.getMessage());
|
||||
exceptionThrown = true;
|
||||
}
|
||||
Assert.assertTrue(exceptionThrown);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOnHeapAggregatable()
|
||||
{
|
||||
T rac = makeRowsAndColumns(MapOfColumnsRowsAndColumns.fromMap(
|
||||
ImmutableMap.of(
|
||||
"incremented", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}),
|
||||
"zeroesOut", new IntArrayColumn(new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0})
|
||||
)
|
||||
));
|
||||
|
||||
validateOnHeapAggregatable("default", new DefaultOnHeapAggregatable(rac));
|
||||
|
||||
OnHeapAggregatable specialized = rac.as(OnHeapAggregatable.class);
|
||||
if (specialized != null) {
|
||||
validateOnHeapAggregatable("specialized", specialized);
|
||||
}
|
||||
}
|
||||
|
||||
private void validateOnHeapAggregatable(String name, OnHeapAggregatable agger)
|
||||
{
|
||||
final ArrayList<Object> results = agger.aggregateAll(Arrays.asList(
|
||||
new LongSumAggregatorFactory("incremented", "incremented"),
|
||||
new LongMaxAggregatorFactory("zeroesOutMax", "zeroesOut"),
|
||||
new LongMinAggregatorFactory("zeroesOutMin", "zeroesOut")
|
||||
));
|
||||
|
||||
Assert.assertEquals(name, 3, results.size());
|
||||
Assert.assertEquals(name, 55L, results.get(0));
|
||||
Assert.assertEquals(name, 82L, results.get(1));
|
||||
Assert.assertEquals(name, -90L, results.get(2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppendableRowsAndColumns()
|
||||
{
|
||||
T rac = makeRowsAndColumns(MapOfColumnsRowsAndColumns.fromMap(
|
||||
ImmutableMap.of(
|
||||
"colA", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}),
|
||||
"colB", new IntArrayColumn(new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0})
|
||||
)
|
||||
));
|
||||
|
||||
validateAppendableRowsAndColumns("default", new AppendableMapOfColumns(rac));
|
||||
|
||||
AppendableRowsAndColumns specialized = rac.as(AppendableRowsAndColumns.class);
|
||||
if (specialized != null) {
|
||||
validateAppendableRowsAndColumns("specialized", specialized);
|
||||
}
|
||||
}
|
||||
|
||||
public void validateAppendableRowsAndColumns(String name, AppendableRowsAndColumns appender)
|
||||
{
|
||||
appender.addColumn("newCol", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}));
|
||||
|
||||
new RowsAndColumnsHelper()
|
||||
.expectColumn("colA", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
.expectColumn("colB", new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0})
|
||||
.expectColumn("newCol", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
.allColumnsRegistered()
|
||||
.validate(name, appender);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* 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.druid.query.rowsandcols.column;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class NullColumnAccessorTest
|
||||
{
|
||||
|
||||
@Test
|
||||
public void testSanity()
|
||||
{
|
||||
NullColumnAccessor accessor = new NullColumnAccessor(10);
|
||||
Assert.assertEquals(10, accessor.numRows());
|
||||
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
Assert.assertTrue(accessor.isNull(i));
|
||||
Assert.assertNull(accessor.getObject(i));
|
||||
Assert.assertEquals(0, accessor.getInt(i));
|
||||
Assert.assertEquals(0, accessor.getLong(i));
|
||||
Assert.assertEquals(0.0, accessor.getFloat(i), 0);
|
||||
Assert.assertEquals(0.0, accessor.getDouble(i), 0);
|
||||
for (int j = 0; j < i; ++j) {
|
||||
Assert.assertEquals(0, accessor.compareCells(j, i));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -357,7 +357,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compare(T o1, T o2)
|
||||
public int compare(Object o1, Object o2)
|
||||
{
|
||||
return delegate.compare(o1, o2);
|
||||
}
|
||||
|
|
|
@ -35,6 +35,8 @@ import org.apache.druid.query.groupby.GroupByQueryEngine;
|
|||
import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
|
||||
import org.apache.druid.query.metadata.SegmentMetadataQueryRunnerFactory;
|
||||
import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||
import org.apache.druid.query.operator.WindowOperatorQuery;
|
||||
import org.apache.druid.query.operator.WindowOperatorQueryQueryRunnerFactory;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
import org.apache.druid.query.scan.ScanQueryRunnerFactory;
|
||||
import org.apache.druid.query.search.SearchQuery;
|
||||
|
@ -56,14 +58,15 @@ public class QueryRunnerFactoryModule extends QueryToolChestModule
|
|||
{
|
||||
private static final Map<Class<? extends Query<?>>, Class<? extends QueryRunnerFactory<?, ?>>> MAPPINGS =
|
||||
ImmutableMap.<Class<? extends Query<?>>, Class<? extends QueryRunnerFactory<?, ?>>>builder()
|
||||
.put(TimeseriesQuery.class, TimeseriesQueryRunnerFactory.class)
|
||||
.put(SearchQuery.class, SearchQueryRunnerFactory.class)
|
||||
.put(TimeBoundaryQuery.class, TimeBoundaryQueryRunnerFactory.class)
|
||||
.put(SegmentMetadataQuery.class, SegmentMetadataQueryRunnerFactory.class)
|
||||
.put(DataSourceMetadataQuery.class, DataSourceMetadataQueryRunnerFactory.class)
|
||||
.put(GroupByQuery.class, GroupByQueryRunnerFactory.class)
|
||||
.put(ScanQuery.class, ScanQueryRunnerFactory.class)
|
||||
.put(SearchQuery.class, SearchQueryRunnerFactory.class)
|
||||
.put(SegmentMetadataQuery.class, SegmentMetadataQueryRunnerFactory.class)
|
||||
.put(TimeBoundaryQuery.class, TimeBoundaryQueryRunnerFactory.class)
|
||||
.put(TimeseriesQuery.class, TimeseriesQueryRunnerFactory.class)
|
||||
.put(TopNQuery.class, TopNQueryRunnerFactory.class)
|
||||
.put(DataSourceMetadataQuery.class, DataSourceMetadataQueryRunnerFactory.class)
|
||||
.put(WindowOperatorQuery.class, WindowOperatorQueryQueryRunnerFactory.class)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
|
|
|
@ -41,6 +41,8 @@ import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
|
|||
import org.apache.druid.query.metadata.SegmentMetadataQueryConfig;
|
||||
import org.apache.druid.query.metadata.SegmentMetadataQueryQueryToolChest;
|
||||
import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||
import org.apache.druid.query.operator.WindowOperatorQuery;
|
||||
import org.apache.druid.query.operator.WindowOperatorQueryQueryToolChest;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
import org.apache.druid.query.scan.ScanQueryConfig;
|
||||
import org.apache.druid.query.scan.ScanQueryQueryToolChest;
|
||||
|
@ -75,14 +77,15 @@ public class QueryToolChestModule implements Module
|
|||
|
||||
public final Map<Class<? extends Query>, Class<? extends QueryToolChest>> mappings =
|
||||
ImmutableMap.<Class<? extends Query>, Class<? extends QueryToolChest>>builder()
|
||||
.put(TimeseriesQuery.class, TimeseriesQueryQueryToolChest.class)
|
||||
.put(SearchQuery.class, SearchQueryQueryToolChest.class)
|
||||
.put(TimeBoundaryQuery.class, TimeBoundaryQueryQueryToolChest.class)
|
||||
.put(SegmentMetadataQuery.class, SegmentMetadataQueryQueryToolChest.class)
|
||||
.put(DataSourceMetadataQuery.class, DataSourceQueryQueryToolChest.class)
|
||||
.put(GroupByQuery.class, GroupByQueryQueryToolChest.class)
|
||||
.put(ScanQuery.class, ScanQueryQueryToolChest.class)
|
||||
.put(SearchQuery.class, SearchQueryQueryToolChest.class)
|
||||
.put(SegmentMetadataQuery.class, SegmentMetadataQueryQueryToolChest.class)
|
||||
.put(TimeBoundaryQuery.class, TimeBoundaryQueryQueryToolChest.class)
|
||||
.put(TimeseriesQuery.class, TimeseriesQueryQueryToolChest.class)
|
||||
.put(TopNQuery.class, TopNQueryQueryToolChest.class)
|
||||
.put(DataSourceMetadataQuery.class, DataSourceQueryQueryToolChest.class)
|
||||
.put(WindowOperatorQuery.class, WindowOperatorQueryQueryToolChest.class)
|
||||
.build();
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,14 +22,14 @@ package org.apache.druid.segment;
|
|||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.segment.join.JoinableFactory;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* A {@link JoinableFactory} for {@link InlineDataSource}.
|
||||
* A {@link SegmentWrangler} for {@link InlineDataSource}.
|
||||
*
|
||||
* It is not valid to pass any other DataSource type to the "getSegmentsForIntervals" method.
|
||||
*/
|
||||
|
@ -38,10 +38,22 @@ public class InlineSegmentWrangler implements SegmentWrangler
|
|||
private static final String SEGMENT_ID = "inline";
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Iterable<Segment> getSegmentsForIntervals(final DataSource dataSource, final Iterable<Interval> intervals)
|
||||
{
|
||||
final InlineDataSource inlineDataSource = (InlineDataSource) dataSource;
|
||||
|
||||
if (inlineDataSource.rowsAreArrayList()) {
|
||||
return Collections.singletonList(
|
||||
new ArrayListSegment<>(
|
||||
SegmentId.dummy(SEGMENT_ID),
|
||||
(ArrayList<Object[]>) inlineDataSource.getRowsAsList(),
|
||||
inlineDataSource.rowAdapter(),
|
||||
inlineDataSource.getRowSignature()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
return Collections.singletonList(
|
||||
new RowBasedSegment<>(
|
||||
SegmentId.dummy(SEGMENT_ID),
|
||||
|
|
|
@ -72,9 +72,9 @@ import java.util.stream.Collectors;
|
|||
|
||||
/**
|
||||
* Query handler for the Broker processes (see CliBroker).
|
||||
*
|
||||
* <p>
|
||||
* This class is responsible for:
|
||||
*
|
||||
* <p>
|
||||
* 1) Running queries on the cluster using its 'clusterClient'
|
||||
* 2) Running queries locally (when all datasources are global) using its 'localClient'
|
||||
* 3) Inlining subqueries if necessary, in service of the above two goals
|
||||
|
@ -133,7 +133,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
|
|||
{
|
||||
this(
|
||||
emitter,
|
||||
(QuerySegmentWalker) clusterClient,
|
||||
clusterClient,
|
||||
(QuerySegmentWalker) localClient,
|
||||
warehouse,
|
||||
joinableFactory,
|
||||
|
@ -291,20 +291,20 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
|
|||
|
||||
/**
|
||||
* Replace QueryDataSources with InlineDataSources when necessary and possible. "Necessary" is defined as:
|
||||
*
|
||||
* <p>
|
||||
* 1) For outermost subqueries: inlining is necessary if the toolchest cannot handle it.
|
||||
* 2) For all other subqueries (e.g. those nested under a join): inlining is always necessary.
|
||||
*
|
||||
* @param dataSource datasource to process.
|
||||
* @param toolChestIfOutermost if provided, and if the provided datasource is a {@link QueryDataSource}, this method
|
||||
* will consider whether the toolchest can handle a subquery on the datasource using
|
||||
* {@link QueryToolChest#canPerformSubquery}. If the toolchest can handle it, then it will
|
||||
* not be inlined. See {@link org.apache.druid.query.groupby.GroupByQueryQueryToolChest}
|
||||
* for an example of a toolchest that can handle subqueries.
|
||||
* @param dataSource datasource to process.
|
||||
* @param toolChestIfOutermost if provided, and if the provided datasource is a {@link QueryDataSource}, this method
|
||||
* will consider whether the toolchest can handle a subquery on the datasource using
|
||||
* {@link QueryToolChest#canPerformSubquery}. If the toolchest can handle it, then it will
|
||||
* not be inlined. See {@link org.apache.druid.query.groupby.GroupByQueryQueryToolChest}
|
||||
* for an example of a toolchest that can handle subqueries.
|
||||
* @param subqueryRowLimitAccumulator an accumulator for tracking the number of accumulated rows in all subqueries
|
||||
* for a particular master query
|
||||
* @param maxSubqueryRows Max rows that all the subqueries generated by a master query can have, combined
|
||||
* @param dryRun if true, does not actually execute any subqueries, but will inline empty result sets.
|
||||
* @param maxSubqueryRows Max rows that all the subqueries generated by a master query can have, combined
|
||||
* @param dryRun if true, does not actually execute any subqueries, but will inline empty result sets.
|
||||
*/
|
||||
@SuppressWarnings({"rawtypes", "unchecked"}) // Subquery, toolchest, runner handling all use raw types
|
||||
private DataSource inlineIfNecessary(
|
||||
|
@ -331,15 +331,18 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
|
|||
current = Iterables.getOnlyElement(current.getChildren());
|
||||
}
|
||||
|
||||
assert !(current instanceof QueryDataSource); // lgtm [java/contradictory-type-checks]
|
||||
if (current instanceof QueryDataSource) {
|
||||
throw new ISE("Got a QueryDataSource[%s], should've walked it away in the loop above.", current);
|
||||
}
|
||||
current = inlineIfNecessary(current, null, subqueryRowLimitAccumulator, maxSubqueryRows, dryRun);
|
||||
|
||||
while (!stack.isEmpty()) {
|
||||
current = stack.pop().withChildren(Collections.singletonList(current));
|
||||
}
|
||||
|
||||
assert current instanceof QueryDataSource;
|
||||
|
||||
if (!(current instanceof QueryDataSource)) {
|
||||
throw new ISE("Should have a QueryDataSource, but got[%s] instead", current);
|
||||
}
|
||||
if (toolChest.canPerformSubquery(((QueryDataSource) current).getQuery())) {
|
||||
return current;
|
||||
} else {
|
||||
|
@ -431,7 +434,9 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
|
|||
.postProcess(
|
||||
objectMapper.convertValue(
|
||||
query.context().getString("postProcessing"),
|
||||
new TypeReference<PostProcessingOperator<T>>() {}
|
||||
new TypeReference<PostProcessingOperator<T>>()
|
||||
{
|
||||
}
|
||||
)
|
||||
)
|
||||
.map(
|
||||
|
@ -572,7 +577,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
|
|||
|
||||
final RowSignature signature = toolChest.resultArraySignature(query);
|
||||
|
||||
final List<Object[]> resultList = new ArrayList<>();
|
||||
final ArrayList<Object[]> resultList = new ArrayList<>();
|
||||
|
||||
toolChest.resultsAsArrays(query, results).accumulate(
|
||||
resultList,
|
||||
|
|
|
@ -93,12 +93,10 @@ public class LocalQuerySegmentWalker implements QuerySegmentWalker
|
|||
|
||||
final AtomicLong cpuAccumulator = new AtomicLong(0L);
|
||||
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = analysis
|
||||
.getDataSource()
|
||||
.createSegmentMapFunction(
|
||||
query,
|
||||
cpuAccumulator
|
||||
);
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn =
|
||||
analysis
|
||||
.getDataSource()
|
||||
.createSegmentMapFunction(query, cpuAccumulator);
|
||||
|
||||
|
||||
final QueryRunnerFactory<T, Query<T>> queryRunnerFactory = conglomerate.findFactory(query);
|
||||
|
|
|
@ -194,9 +194,11 @@ public class ServerManager implements QuerySegmentWalker
|
|||
} else {
|
||||
return new ReportTimelineMissingSegmentQueryRunner<>(Lists.newArrayList(specs));
|
||||
}
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = query.getDataSource().createSegmentMapFunction(query, cpuTimeAccumulator);
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn =
|
||||
query.getDataSource()
|
||||
.createSegmentMapFunction(query, cpuTimeAccumulator);
|
||||
|
||||
// We compute the join cache key here itself so it doesn't need to be re-computed for every segment
|
||||
// We compute the datasource's cache key here itself so it doesn't need to be re-computed for every segment
|
||||
final Optional<byte[]> cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey());
|
||||
|
||||
final FunctionalIterable<QueryRunner<T>> queryRunners = FunctionalIterable
|
||||
|
|
|
@ -1391,7 +1391,6 @@ public class ClientQuerySegmentWalkerTest
|
|||
.put(ARRAY, makeTimeline(ARRAY, ARRAY_INLINE))
|
||||
.put(ARRAY_UNKNOWN, makeTimeline(ARRAY_UNKNOWN, ARRAY_INLINE_UNKNOWN))
|
||||
.build(),
|
||||
joinableFactoryWrapper,
|
||||
conglomerate,
|
||||
schedulerForTest
|
||||
),
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue