mirror of
https://github.com/apache/druid.git
synced 2025-02-17 07:25:02 +00:00
Rename ObjectValueSelector.get() to getObject(); Add getObject() and classOfObject() to ColumnValueSelector (#4801)
This commit is contained in:
parent
164c73f2b2
commit
88e9a80636
@ -65,7 +65,7 @@ public class TimestampAggregator implements Aggregator
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
Long value = TimestampAggregatorFactory.convertLong(timestampSpec, selector.get());
|
||||
Long value = TimestampAggregatorFactory.convertLong(timestampSpec, selector.getObject());
|
||||
|
||||
if (value != null) {
|
||||
most = comparator.compare(most, value) > 0 ? most : value;
|
||||
|
@ -106,7 +106,7 @@ public class TimestampAggregatorFactory extends AggregatorFactory
|
||||
private long getTimestamp(ColumnValueSelector selector)
|
||||
{
|
||||
if (selector instanceof ObjectColumnSelector) {
|
||||
Object input = ((ObjectColumnSelector) selector).get();
|
||||
Object input = ((ObjectColumnSelector) selector).getObject();
|
||||
return convertLong(timestampSpec, input);
|
||||
} else {
|
||||
return selector.getLong();
|
||||
|
@ -54,7 +54,7 @@ public class TimestampBufferAggregator implements BufferAggregator
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
Long newTime = TimestampAggregatorFactory.convertLong(timestampSpec, selector.get());
|
||||
Long newTime = TimestampAggregatorFactory.convertLong(timestampSpec, selector.getObject());
|
||||
if (newTime != null) {
|
||||
long prev = buf.getLong(position);
|
||||
buf.putLong(position, comparator.compare(prev, newTime) > 0 ? prev : newTime);
|
||||
|
@ -81,7 +81,7 @@ public class MapVirtualColumn implements VirtualColumn
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map get()
|
||||
public Map getObject()
|
||||
{
|
||||
final IndexedInts keyIndices = keySelector.getRow();
|
||||
final IndexedInts valueIndices = valueSelector.getRow();
|
||||
@ -116,7 +116,7 @@ public class MapVirtualColumn implements VirtualColumn
|
||||
}
|
||||
|
||||
@Override
|
||||
public String get()
|
||||
public String getObject()
|
||||
{
|
||||
final IndexedInts keyIndices = keySelector.getRow();
|
||||
final IndexedInts valueIndices = valueSelector.getRow();
|
||||
@ -142,7 +142,7 @@ public class MapVirtualColumn implements VirtualColumn
|
||||
}
|
||||
|
||||
@Override
|
||||
public String get()
|
||||
public String getObject()
|
||||
{
|
||||
final IndexedInts keyIndices = keySelector.getRow();
|
||||
final IndexedInts valueIndices = valueSelector.getRow();
|
||||
|
@ -43,7 +43,7 @@ public class SketchAggregator implements Aggregator
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
Object update = selector.get();
|
||||
Object update = selector.getObject();
|
||||
if (update == null) {
|
||||
return;
|
||||
}
|
||||
|
@ -123,7 +123,7 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory
|
||||
public void fold(ColumnValueSelector selector)
|
||||
{
|
||||
@SuppressWarnings("unchecked")
|
||||
SketchHolder other = ((ObjectColumnSelector<SketchHolder>) selector).get();
|
||||
SketchHolder other = ((ObjectColumnSelector<SketchHolder>) selector).getObject();
|
||||
// SketchAggregatorFactory.combine() delegates to SketchHolder.combine() and it doesn't check for nulls, so we
|
||||
// neither.
|
||||
other.updateUnion(union);
|
||||
@ -138,7 +138,7 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public SketchHolder get()
|
||||
public SketchHolder getObject()
|
||||
{
|
||||
return combined;
|
||||
}
|
||||
|
@ -58,7 +58,7 @@ public class SketchBufferAggregator implements BufferAggregator
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
Object update = selector.get();
|
||||
Object update = selector.getObject();
|
||||
if (update == null) {
|
||||
return;
|
||||
}
|
||||
|
@ -128,7 +128,7 @@ public class ApproximateHistogramAggregatorFactory extends AggregatorFactory
|
||||
public void reset(ColumnValueSelector selector)
|
||||
{
|
||||
@SuppressWarnings("unchecked")
|
||||
ApproximateHistogram first = ((ObjectColumnSelector<ApproximateHistogram>) selector).get();
|
||||
ApproximateHistogram first = ((ObjectColumnSelector<ApproximateHistogram>) selector).getObject();
|
||||
combined.copy(first);
|
||||
}
|
||||
|
||||
@ -136,7 +136,7 @@ public class ApproximateHistogramAggregatorFactory extends AggregatorFactory
|
||||
public void fold(ColumnValueSelector selector)
|
||||
{
|
||||
@SuppressWarnings("unchecked")
|
||||
ApproximateHistogram other = ((ObjectColumnSelector<ApproximateHistogram>) selector).get();
|
||||
ApproximateHistogram other = ((ObjectColumnSelector<ApproximateHistogram>) selector).getObject();
|
||||
combined.foldFast(other);
|
||||
}
|
||||
|
||||
@ -148,7 +148,7 @@ public class ApproximateHistogramAggregatorFactory extends AggregatorFactory
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ApproximateHistogram get()
|
||||
public ApproximateHistogram getObject()
|
||||
{
|
||||
return combined;
|
||||
}
|
||||
|
@ -54,7 +54,7 @@ public class ApproximateHistogramFoldingAggregator implements Aggregator
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
ApproximateHistogram h = selector.get();
|
||||
ApproximateHistogram h = selector.getObject();
|
||||
if (h == null) {
|
||||
return;
|
||||
}
|
||||
|
@ -70,7 +70,7 @@ public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHis
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApproximateHistogram get()
|
||||
public ApproximateHistogram getObject()
|
||||
{
|
||||
return new ApproximateHistogram(0);
|
||||
}
|
||||
@ -111,7 +111,7 @@ public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHis
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApproximateHistogram get()
|
||||
public ApproximateHistogram getObject()
|
||||
{
|
||||
return new ApproximateHistogram(0);
|
||||
}
|
||||
|
@ -71,7 +71,7 @@ public class ApproximateHistogramFoldingBufferAggregator implements BufferAggreg
|
||||
ApproximateHistogram h0 = ApproximateHistogram.fromBytesDense(mutationBuffer);
|
||||
h0.setLowerLimit(lowerLimit);
|
||||
h0.setUpperLimit(upperLimit);
|
||||
ApproximateHistogram hNext = selector.get();
|
||||
ApproximateHistogram hNext = selector.getObject();
|
||||
h0.foldFast(hNext, tmpBufferP, tmpBufferB);
|
||||
|
||||
mutationBuffer.position(position);
|
||||
|
@ -116,7 +116,7 @@ public abstract class VarianceAggregator implements Aggregator
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
VarianceAggregatorCollector.combineValues(holder, selector.get());
|
||||
VarianceAggregatorCollector.combineValues(holder, selector.getObject());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -157,7 +157,7 @@ public class VarianceAggregatorFactory extends AggregatorFactory
|
||||
public void reset(ColumnValueSelector selector)
|
||||
{
|
||||
@SuppressWarnings("unchecked")
|
||||
VarianceAggregatorCollector first = ((ObjectColumnSelector<VarianceAggregatorCollector>) selector).get();
|
||||
VarianceAggregatorCollector first = ((ObjectColumnSelector<VarianceAggregatorCollector>) selector).getObject();
|
||||
combined.copyFrom(first);
|
||||
}
|
||||
|
||||
@ -165,7 +165,7 @@ public class VarianceAggregatorFactory extends AggregatorFactory
|
||||
public void fold(ColumnValueSelector selector)
|
||||
{
|
||||
@SuppressWarnings("unchecked")
|
||||
VarianceAggregatorCollector other = ((ObjectColumnSelector<VarianceAggregatorCollector>) selector).get();
|
||||
VarianceAggregatorCollector other = ((ObjectColumnSelector<VarianceAggregatorCollector>) selector).getObject();
|
||||
combined.fold(other);
|
||||
}
|
||||
|
||||
@ -176,7 +176,7 @@ public class VarianceAggregatorFactory extends AggregatorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public VarianceAggregatorCollector get()
|
||||
public VarianceAggregatorCollector getObject()
|
||||
{
|
||||
return combined;
|
||||
}
|
||||
|
@ -162,7 +162,7 @@ public abstract class VarianceBufferAggregator implements BufferAggregator
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
VarianceAggregatorCollector holder2 = (VarianceAggregatorCollector) selector.get();
|
||||
VarianceAggregatorCollector holder2 = (VarianceAggregatorCollector) selector.getObject();
|
||||
|
||||
long count = buf.getLong(position + COUNT_OFFSET);
|
||||
if (count == 0) {
|
||||
|
@ -163,7 +163,7 @@ public class VarianceAggregatorCollectorTest
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
return v;
|
||||
}
|
||||
|
@ -38,14 +38,14 @@ import io.druid.segment.ColumnValueSelector;
|
||||
* @see DoubleAggregateCombiner
|
||||
* @see ObjectAggregateCombiner
|
||||
*/
|
||||
public interface AggregateCombiner extends ColumnValueSelector
|
||||
public interface AggregateCombiner<T> extends ColumnValueSelector<T>
|
||||
{
|
||||
/**
|
||||
* Resets this AggregateCombiner's state value to the value of the given selector, e. g. after calling this method
|
||||
* combiner.get*() should return the same value as selector.get*().
|
||||
*
|
||||
* If the selector is an {@link io.druid.segment.ObjectColumnSelector}, the object returned from {@link
|
||||
* io.druid.segment.ObjectColumnSelector#get()} must not be modified, and must not become a subject for modification
|
||||
* io.druid.segment.ObjectColumnSelector#getObject()} must not be modified, and must not become a subject for modification
|
||||
* during subsequent {@link #fold} calls.
|
||||
*/
|
||||
void reset(ColumnValueSelector selector);
|
||||
@ -57,7 +57,7 @@ public interface AggregateCombiner extends ColumnValueSelector
|
||||
* aggregatorFactory.combine(combiner.get*(), selector.get*())} call.
|
||||
*
|
||||
* Unlike {@link AggregatorFactory#combine}, if the selector is an {@link io.druid.segment.ObjectColumnSelector}, the
|
||||
* object returned from {@link io.druid.segment.ObjectColumnSelector#get()} must not be modified, and must not become
|
||||
* object returned from {@link io.druid.segment.ObjectColumnSelector#getObject()} must not be modified, and must not become
|
||||
* a subject for modification during subsequent fold() calls.
|
||||
*
|
||||
* Since the state of AggregateCombiner is undefined before {@link #reset} is ever called on it, the effects of
|
||||
|
@ -25,7 +25,7 @@ import io.druid.segment.DoubleColumnSelector;
|
||||
/**
|
||||
* Specialization of {@link AggregateCombiner} for primitive double aggregations.
|
||||
*/
|
||||
public abstract class DoubleAggregateCombiner implements AggregateCombiner, DoubleColumnSelector
|
||||
public abstract class DoubleAggregateCombiner implements AggregateCombiner<Double>, DoubleColumnSelector
|
||||
{
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
|
@ -104,7 +104,7 @@ public class HistogramAggregatorFactory extends AggregatorFactory
|
||||
public void reset(ColumnValueSelector selector)
|
||||
{
|
||||
@SuppressWarnings("unchecked")
|
||||
Histogram first = ((ObjectColumnSelector<Histogram>) selector).get();
|
||||
Histogram first = ((ObjectColumnSelector<Histogram>) selector).getObject();
|
||||
if (combined == null) {
|
||||
combined = new Histogram(first);
|
||||
} else {
|
||||
@ -116,7 +116,7 @@ public class HistogramAggregatorFactory extends AggregatorFactory
|
||||
public void fold(ColumnValueSelector selector)
|
||||
{
|
||||
@SuppressWarnings("unchecked")
|
||||
Histogram other = ((ObjectColumnSelector<Histogram>) selector).get();
|
||||
Histogram other = ((ObjectColumnSelector<Histogram>) selector).getObject();
|
||||
combined.fold(other);
|
||||
}
|
||||
|
||||
@ -128,7 +128,7 @@ public class HistogramAggregatorFactory extends AggregatorFactory
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Histogram get()
|
||||
public Histogram getObject()
|
||||
{
|
||||
return combined;
|
||||
}
|
||||
|
@ -347,7 +347,7 @@ public class JavaScriptAggregatorFactory extends AggregatorFactory
|
||||
for (int i = 0; i < size; i++) {
|
||||
final ObjectColumnSelector selector = selectorList[i];
|
||||
if (selector != null) {
|
||||
final Object arg = selector.get();
|
||||
final Object arg = selector.getObject();
|
||||
if (arg != null && arg.getClass().isArray()) {
|
||||
// Context.javaToJS on an array sort of works, although it returns false for Array.isArray(...) and
|
||||
// may have other issues too. Let's just copy the array and wrap that.
|
||||
|
@ -25,7 +25,7 @@ import io.druid.segment.LongColumnSelector;
|
||||
/**
|
||||
* Specialization of {@link AggregateCombiner} for primitive long aggregations.
|
||||
*/
|
||||
public abstract class LongAggregateCombiner implements AggregateCombiner, LongColumnSelector
|
||||
public abstract class LongAggregateCombiner implements AggregateCombiner<Long>, LongColumnSelector
|
||||
{
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
|
@ -24,6 +24,6 @@ import io.druid.segment.ObjectColumnSelector;
|
||||
/**
|
||||
* Specialization of {@link AggregateCombiner} for object aggregations.
|
||||
*/
|
||||
public abstract class ObjectAggregateCombiner<T> implements AggregateCombiner, ObjectColumnSelector<T>
|
||||
public abstract class ObjectAggregateCombiner<T> implements AggregateCombiner<T>, ObjectColumnSelector<T>
|
||||
{
|
||||
}
|
||||
|
@ -42,7 +42,7 @@ public final class HyperLogLogCollectorAggregateCombiner extends ObjectAggregate
|
||||
public void fold(ColumnValueSelector selector)
|
||||
{
|
||||
@SuppressWarnings("unchecked")
|
||||
HyperLogLogCollector other = ((ObjectColumnSelector<HyperLogLogCollector>) selector).get();
|
||||
HyperLogLogCollector other = ((ObjectColumnSelector<HyperLogLogCollector>) selector).getObject();
|
||||
if (other == null) {
|
||||
return;
|
||||
}
|
||||
@ -60,7 +60,7 @@ public final class HyperLogLogCollectorAggregateCombiner extends ObjectAggregate
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public HyperLogLogCollector get()
|
||||
public HyperLogLogCollector getObject()
|
||||
{
|
||||
return combined;
|
||||
}
|
||||
|
@ -124,7 +124,7 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.get();
|
||||
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.getObject();
|
||||
if (pair.lhs < firstTime) {
|
||||
firstTime = pair.lhs;
|
||||
firstValue = pair.rhs;
|
||||
@ -142,7 +142,7 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.get();
|
||||
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.getObject();
|
||||
long firstTime = buf.getLong(position);
|
||||
if (pair.lhs < firstTime) {
|
||||
buf.putLong(position, pair.lhs);
|
||||
|
@ -124,7 +124,7 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.get();
|
||||
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.getObject();
|
||||
if (pair.lhs < firstTime) {
|
||||
firstTime = pair.lhs;
|
||||
firstValue = pair.rhs;
|
||||
@ -142,7 +142,7 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.get();
|
||||
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.getObject();
|
||||
long firstTime = buf.getLong(position);
|
||||
if (pair.lhs < firstTime) {
|
||||
buf.putLong(position, pair.lhs);
|
||||
|
@ -117,7 +117,7 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.get();
|
||||
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.getObject();
|
||||
if (pair.lhs < firstTime) {
|
||||
firstTime = pair.lhs;
|
||||
firstValue = pair.rhs;
|
||||
@ -135,7 +135,7 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.get();
|
||||
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.getObject();
|
||||
long firstTime = buf.getLong(position);
|
||||
if (pair.lhs < firstTime) {
|
||||
buf.putLong(position, pair.lhs);
|
||||
|
@ -43,7 +43,7 @@ public class HyperUniquesAggregator implements Aggregator
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
collector.fold((HyperLogLogCollector) selector.get());
|
||||
collector.fold((HyperLogLogCollector) selector.getObject());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -51,7 +51,7 @@ public class HyperUniquesBufferAggregator implements BufferAggregator
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
HyperLogLogCollector collector = (HyperLogLogCollector) selector.get();
|
||||
HyperLogLogCollector collector = (HyperLogLogCollector) selector.getObject();
|
||||
|
||||
if (collector == null) {
|
||||
return;
|
||||
|
@ -115,7 +115,7 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.get();
|
||||
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.getObject();
|
||||
if (pair.lhs >= lastTime) {
|
||||
lastTime = pair.lhs;
|
||||
lastValue = pair.rhs;
|
||||
@ -133,7 +133,7 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.get();
|
||||
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.getObject();
|
||||
long lastTime = buf.getLong(position);
|
||||
if (pair.lhs >= lastTime) {
|
||||
buf.putLong(position, pair.lhs);
|
||||
|
@ -115,7 +115,7 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.get();
|
||||
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.getObject();
|
||||
if (pair.lhs >= lastTime) {
|
||||
lastTime = pair.lhs;
|
||||
lastValue = pair.rhs;
|
||||
@ -133,7 +133,7 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.get();
|
||||
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.getObject();
|
||||
long lastTime = buf.getLong(position);
|
||||
if (pair.lhs >= lastTime) {
|
||||
buf.putLong(position, pair.lhs);
|
||||
|
@ -113,7 +113,7 @@ public class LongLastAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.get();
|
||||
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.getObject();
|
||||
if (pair.lhs >= lastTime) {
|
||||
lastTime = pair.lhs;
|
||||
lastValue = pair.rhs;
|
||||
@ -131,7 +131,7 @@ public class LongLastAggregatorFactory extends AggregatorFactory
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.get();
|
||||
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.getObject();
|
||||
long lastTime = buf.getLong(position);
|
||||
if (pair.lhs >= lastTime) {
|
||||
buf.putLong(position, pair.lhs);
|
||||
|
@ -437,7 +437,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
return row.get().getTimestampFromEpoch();
|
||||
}
|
||||
@ -452,7 +452,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
return row.get().getRaw(columnName);
|
||||
}
|
||||
|
@ -241,9 +241,9 @@ public class ScanQueryEngine
|
||||
final Object value;
|
||||
|
||||
if (legacy && allColumns.get(i).equals(LEGACY_TIMESTAMP_KEY)) {
|
||||
value = DateTimes.utc((long) selector.get());
|
||||
value = DateTimes.utc((long) selector.getObject());
|
||||
} else {
|
||||
value = selector == null ? null : selector.get();
|
||||
value = selector == null ? null : selector.getObject();
|
||||
}
|
||||
|
||||
return value;
|
||||
|
@ -315,7 +315,7 @@ public class SelectQueryEngine
|
||||
if (selector == null) {
|
||||
theEvent.put(metric, null);
|
||||
} else {
|
||||
theEvent.put(metric, selector.get());
|
||||
theEvent.put(metric, selector.getObject());
|
||||
}
|
||||
}
|
||||
return theEvent;
|
||||
|
@ -39,9 +39,9 @@ import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.data.Offset;
|
||||
import io.druid.segment.historical.HistoricalColumnSelector;
|
||||
import io.druid.segment.historical.HistoricalCursor;
|
||||
import io.druid.segment.historical.HistoricalDimensionSelector;
|
||||
import io.druid.segment.historical.HistoricalColumnSelector;
|
||||
import io.druid.segment.historical.SingleValueHistoricalDimensionSelector;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -19,6 +19,8 @@
|
||||
|
||||
package io.druid.segment;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* Base type for interfaces that manage column value selection, e.g. DimensionSelector, LongColumnSelector
|
||||
*
|
||||
@ -26,11 +28,16 @@ package io.druid.segment;
|
||||
* Druid: Sum, Min, Max, etc: {@link #getFloat()}, {@link #getDouble()} and {@link #getLong()} to support "polymorphic"
|
||||
* rollup aggregation during index merging.
|
||||
*/
|
||||
public interface ColumnValueSelector
|
||||
public interface ColumnValueSelector<T>
|
||||
{
|
||||
float getFloat();
|
||||
|
||||
double getDouble();
|
||||
|
||||
long getLong();
|
||||
|
||||
@Nullable
|
||||
T getObject();
|
||||
|
||||
Class<T> classOfObject();
|
||||
}
|
||||
|
@ -147,4 +147,24 @@ public interface DimensionSelector extends ColumnValueSelector, HotLoopCallee
|
||||
{
|
||||
throw new UnsupportedOperationException("DimensionSelector cannot be operated as numeric ColumnValueSelector");
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated always throws {@link UnsupportedOperationException}
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
default Object getObject()
|
||||
{
|
||||
throw new UnsupportedOperationException("DimensionSelector cannot be operated as object ColumnValueSelector");
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated always throws {@link UnsupportedOperationException}
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
default Class classOfObject()
|
||||
{
|
||||
throw new UnsupportedOperationException("DimensionSelector cannot be operated as object ColumnValueSelector");
|
||||
}
|
||||
}
|
||||
|
@ -23,7 +23,7 @@ import io.druid.query.monomorphicprocessing.CalledFromHotLoop;
|
||||
import io.druid.query.monomorphicprocessing.HotLoopCallee;
|
||||
|
||||
|
||||
public interface DoubleColumnSelector extends ColumnValueSelector, HotLoopCallee
|
||||
public interface DoubleColumnSelector extends ColumnValueSelector<Double>, HotLoopCallee
|
||||
{
|
||||
@CalledFromHotLoop
|
||||
@Override
|
||||
@ -54,4 +54,28 @@ public interface DoubleColumnSelector extends ColumnValueSelector, HotLoopCallee
|
||||
{
|
||||
return (long) getDouble();
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated This method is marked as deprecated in DoubleColumnSelector to minimize the probability of accidential
|
||||
* calling. "Polymorphism" of DoubleColumnSelector should be used only when operating on {@link ColumnValueSelector}
|
||||
* objects.
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
default Double getObject()
|
||||
{
|
||||
return getDouble();
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated This method is marked as deprecated in DoubleColumnSelector to minimize the probability of accidential
|
||||
* calling. "Polymorphism" of DoubleColumnSelector should be used only when operating on {@link ColumnValueSelector}
|
||||
* objects.
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
default Class<Double> classOfObject()
|
||||
{
|
||||
return Double.class;
|
||||
}
|
||||
}
|
||||
|
@ -27,7 +27,7 @@ import io.druid.query.monomorphicprocessing.HotLoopCallee;
|
||||
* FloatColumnSelector has a handle onto some other stateful object (e.g. an Offset) which is changing between calls
|
||||
* to get() (though, that doesn't have to be the case if you always want the same value...).
|
||||
*/
|
||||
public interface FloatColumnSelector extends ColumnValueSelector, HotLoopCallee
|
||||
public interface FloatColumnSelector extends ColumnValueSelector<Float>, HotLoopCallee
|
||||
{
|
||||
@CalledFromHotLoop
|
||||
@Override
|
||||
@ -58,4 +58,28 @@ public interface FloatColumnSelector extends ColumnValueSelector, HotLoopCallee
|
||||
{
|
||||
return (long) getFloat();
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated This method is marked as deprecated in FloatColumnSelector to minimize the probability of accidential
|
||||
* calling. "Polymorphism" of FloatColumnSelector should be used only when operating on {@link ColumnValueSelector}
|
||||
* objects.
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
default Float getObject()
|
||||
{
|
||||
return getFloat();
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated This method is marked as deprecated in FloatColumnSelector to minimize the probability of accidential
|
||||
* calling. "Polymorphism" of FloatColumnSelector should be used only when operating on {@link ColumnValueSelector}
|
||||
* objects.
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
default Class<Float> classOfObject()
|
||||
{
|
||||
return Float.class;
|
||||
}
|
||||
}
|
||||
|
@ -24,7 +24,7 @@ import io.druid.query.monomorphicprocessing.HotLoopCallee;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface LongColumnSelector extends ColumnValueSelector, HotLoopCallee
|
||||
public interface LongColumnSelector extends ColumnValueSelector<Long>, HotLoopCallee
|
||||
{
|
||||
@CalledFromHotLoop
|
||||
@Override
|
||||
@ -55,4 +55,28 @@ public interface LongColumnSelector extends ColumnValueSelector, HotLoopCallee
|
||||
{
|
||||
return (double) getLong();
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated This method is marked as deprecated in LongColumnSelector to minimize the probability of accidential
|
||||
* calling. "Polymorphism" of LongColumnSelector should be used only when operating on {@link ColumnValueSelector}
|
||||
* objects.
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
default Long getObject()
|
||||
{
|
||||
return getLong();
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated This method is marked as deprecated in LongColumnSelector to minimize the probability of accidential
|
||||
* calling. "Polymorphism" of LongColumnSelector should be used only when operating on {@link ColumnValueSelector}
|
||||
* objects.
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
default Class<Long> classOfObject()
|
||||
{
|
||||
return Long.class;
|
||||
}
|
||||
}
|
||||
|
@ -39,7 +39,7 @@ public final class NullStringObjectColumnSelector implements ObjectColumnSelecto
|
||||
}
|
||||
|
||||
@Override
|
||||
public String get()
|
||||
public String getObject()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
@ -19,19 +19,8 @@
|
||||
|
||||
package io.druid.segment;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
public interface ObjectColumnSelector<T> extends ColumnValueSelector
|
||||
public interface ObjectColumnSelector<T> extends ColumnValueSelector<T>
|
||||
{
|
||||
public Class<T> classOfObject();
|
||||
|
||||
/**
|
||||
* This method is not annotated with {@link io.druid.query.monomorphicprocessing.CalledFromHotLoop}, because
|
||||
* ObjectColumnSelector doesn't extend {@link io.druid.query.monomorphicprocessing.HotLoopCallee} yet. If it will,
|
||||
* this method should be annotated.
|
||||
*/
|
||||
@Nullable
|
||||
public T get();
|
||||
|
||||
/**
|
||||
* @deprecated This method is marked as deprecated in ObjectColumnSelector to minimize the probability of accidential
|
||||
@ -42,7 +31,7 @@ public interface ObjectColumnSelector<T> extends ColumnValueSelector
|
||||
@Override
|
||||
default float getFloat()
|
||||
{
|
||||
T value = get();
|
||||
T value = getObject();
|
||||
if (value == null) {
|
||||
return 0;
|
||||
}
|
||||
@ -58,7 +47,7 @@ public interface ObjectColumnSelector<T> extends ColumnValueSelector
|
||||
@Override
|
||||
default double getDouble()
|
||||
{
|
||||
T value = get();
|
||||
T value = getObject();
|
||||
if (value == null) {
|
||||
return 0;
|
||||
}
|
||||
@ -74,7 +63,7 @@ public interface ObjectColumnSelector<T> extends ColumnValueSelector
|
||||
@Override
|
||||
default long getLong()
|
||||
{
|
||||
T value = get();
|
||||
T value = getObject();
|
||||
if (value == null) {
|
||||
return 0;
|
||||
}
|
||||
|
@ -252,7 +252,7 @@ class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public Float get()
|
||||
public Float getObject()
|
||||
{
|
||||
return columnVals.getFloatSingleValueRow(offset.getOffset());
|
||||
}
|
||||
@ -268,7 +268,7 @@ class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public Double get()
|
||||
public Double getObject()
|
||||
{
|
||||
return columnVals.getDoubleSingleValueRow(offset.getOffset());
|
||||
}
|
||||
@ -284,7 +284,7 @@ class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long get()
|
||||
public Long getObject()
|
||||
{
|
||||
return columnVals.getLongSingleValueRow(offset.getOffset());
|
||||
}
|
||||
@ -300,7 +300,7 @@ class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public String get()
|
||||
public String getObject()
|
||||
{
|
||||
return columnVals.getStringSingleValueRow(offset.getOffset());
|
||||
}
|
||||
@ -321,7 +321,7 @@ class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
final IndexedInts multiValueRow = columnVals.getMultiValueRow(offset.getOffset());
|
||||
if (multiValueRow.size() == 0) {
|
||||
@ -347,7 +347,7 @@ class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public String get()
|
||||
public String getObject()
|
||||
{
|
||||
return columnVals.lookupName(columnVals.getSingleValueRow(offset.getOffset()));
|
||||
}
|
||||
@ -365,7 +365,7 @@ class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
return columnVals.getRowValue(offset.getOffset());
|
||||
}
|
||||
|
@ -22,7 +22,7 @@ package io.druid.segment;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.historical.HistoricalColumnSelector;
|
||||
|
||||
public final class ZeroFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector
|
||||
public final class ZeroFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector<Float>
|
||||
{
|
||||
private static final ZeroFloatColumnSelector INSTANCE = new ZeroFloatColumnSelector();
|
||||
|
||||
|
@ -38,7 +38,7 @@ public interface IndexedDoubles extends Closeable
|
||||
|
||||
default DoubleColumnSelector makeDoubleColumnSelector(ReadableOffset offset)
|
||||
{
|
||||
class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector
|
||||
class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector<Double>
|
||||
{
|
||||
@Override
|
||||
public double getDouble()
|
||||
@ -64,7 +64,7 @@ public interface IndexedDoubles extends Closeable
|
||||
|
||||
default FloatColumnSelector makeFloatColumnSelector(ReadableOffset offset)
|
||||
{
|
||||
class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector
|
||||
class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector<Float>
|
||||
{
|
||||
@Override
|
||||
public float getFloat()
|
||||
@ -90,7 +90,7 @@ public interface IndexedDoubles extends Closeable
|
||||
|
||||
default LongColumnSelector makeLongColumnSelector(ReadableOffset offset)
|
||||
{
|
||||
class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector
|
||||
class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector<Long>
|
||||
{
|
||||
@Override
|
||||
public long getLong()
|
||||
|
@ -41,7 +41,7 @@ public interface IndexedFloats extends Closeable
|
||||
|
||||
default FloatColumnSelector makeFloatColumnSelector(ReadableOffset offset)
|
||||
{
|
||||
class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector
|
||||
class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector<Float>
|
||||
{
|
||||
@Override
|
||||
public float getFloat()
|
||||
@ -67,7 +67,7 @@ public interface IndexedFloats extends Closeable
|
||||
|
||||
default DoubleColumnSelector makeDoubleColumnSelector(ReadableOffset offset)
|
||||
{
|
||||
class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector
|
||||
class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector<Double>
|
||||
{
|
||||
@Override
|
||||
public double getDouble()
|
||||
@ -93,7 +93,7 @@ public interface IndexedFloats extends Closeable
|
||||
|
||||
default LongColumnSelector makeLongColumnSelector(ReadableOffset offset)
|
||||
{
|
||||
class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector
|
||||
class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector<Long>
|
||||
{
|
||||
@Override
|
||||
public long getLong()
|
||||
|
@ -41,7 +41,7 @@ public interface IndexedLongs extends Closeable
|
||||
|
||||
default LongColumnSelector makeLongColumnSelector(ReadableOffset offset)
|
||||
{
|
||||
class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector
|
||||
class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector<Long>
|
||||
{
|
||||
@Override
|
||||
public long getLong()
|
||||
@ -67,7 +67,7 @@ public interface IndexedLongs extends Closeable
|
||||
|
||||
default FloatColumnSelector makeFloatColumnSelector(ReadableOffset offset)
|
||||
{
|
||||
class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector
|
||||
class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector<Float>
|
||||
{
|
||||
@Override
|
||||
public float getFloat()
|
||||
@ -93,7 +93,7 @@ public interface IndexedLongs extends Closeable
|
||||
|
||||
default DoubleColumnSelector makeDoubleColumnSelector(ReadableOffset offset)
|
||||
{
|
||||
class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector
|
||||
class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector<Double>
|
||||
{
|
||||
@Override
|
||||
public double getDouble()
|
||||
|
@ -22,7 +22,7 @@ package io.druid.segment.historical;
|
||||
import io.druid.query.monomorphicprocessing.CalledFromHotLoop;
|
||||
import io.druid.segment.ColumnValueSelector;
|
||||
|
||||
public interface HistoricalColumnSelector extends ColumnValueSelector
|
||||
public interface HistoricalColumnSelector<T> extends ColumnValueSelector<T>
|
||||
{
|
||||
@CalledFromHotLoop
|
||||
double getDouble(int offset);
|
||||
|
@ -166,7 +166,7 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
return extractor.extractValue(in.get(), column);
|
||||
}
|
||||
|
@ -260,7 +260,7 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long get()
|
||||
public Long getObject()
|
||||
{
|
||||
return timeAndDimsHolder.getKey().getTimestamp();
|
||||
}
|
||||
@ -280,7 +280,7 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
return index.getMetricObjectValue(
|
||||
timeAndDimsHolder.getValue(),
|
||||
@ -308,7 +308,7 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
IncrementalIndex.TimeAndDims key = timeAndDimsHolder.getKey();
|
||||
if (key == null) {
|
||||
|
@ -119,11 +119,11 @@ public class ExpressionObjectSelector implements ObjectColumnSelector<ExprEval>
|
||||
final Class<?> clazz = selector.classOfObject();
|
||||
if (Number.class.isAssignableFrom(clazz) || String.class.isAssignableFrom(clazz)) {
|
||||
// Number, String supported as-is.
|
||||
return selector::get;
|
||||
return selector::getObject;
|
||||
} else if (clazz.isAssignableFrom(Number.class) || clazz.isAssignableFrom(String.class)) {
|
||||
// Might be Numbers and Strings. Use a selector that double-checks.
|
||||
return () -> {
|
||||
final Object val = selector.get();
|
||||
final Object val = selector.getObject();
|
||||
if (val instanceof Number || val instanceof String) {
|
||||
return val;
|
||||
} else {
|
||||
@ -143,7 +143,7 @@ public class ExpressionObjectSelector implements ObjectColumnSelector<ExprEval>
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExprEval get()
|
||||
public ExprEval getObject()
|
||||
{
|
||||
return expression.eval(bindings);
|
||||
}
|
||||
|
@ -57,7 +57,7 @@ public class ExpressionSelectors
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
final ExprEval exprEval = baseSelector.get();
|
||||
final ExprEval exprEval = baseSelector.getObject();
|
||||
return exprEval.isNull() ? nullValue : exprEval.asLong();
|
||||
}
|
||||
|
||||
@ -82,7 +82,7 @@ public class ExpressionSelectors
|
||||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
final ExprEval exprEval = baseSelector.get();
|
||||
final ExprEval exprEval = baseSelector.getObject();
|
||||
return exprEval.isNull() ? nullValue : (float) exprEval.asDouble();
|
||||
}
|
||||
|
||||
@ -107,7 +107,7 @@ public class ExpressionSelectors
|
||||
@Override
|
||||
public double getDouble()
|
||||
{
|
||||
final ExprEval exprEval = baseSelector.get();
|
||||
final ExprEval exprEval = baseSelector.getObject();
|
||||
return exprEval.isNull() ? nullValue : exprEval.asDouble();
|
||||
}
|
||||
|
||||
@ -134,7 +134,7 @@ public class ExpressionSelectors
|
||||
@Override
|
||||
protected String getValue()
|
||||
{
|
||||
return Strings.emptyToNull(baseSelector.get().asString());
|
||||
return Strings.emptyToNull(baseSelector.getObject().asString());
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -150,7 +150,7 @@ public class ExpressionSelectors
|
||||
@Override
|
||||
protected String getValue()
|
||||
{
|
||||
return extractionFn.apply(Strings.emptyToNull(baseSelector.get().asString()));
|
||||
return extractionFn.apply(Strings.emptyToNull(baseSelector.getObject().asString()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -101,9 +101,9 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
return baseSelector.get().value();
|
||||
return baseSelector.getObject().value();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -36,7 +36,7 @@ public class MetricSelectorUtils
|
||||
}
|
||||
|
||||
@Override
|
||||
public Float get()
|
||||
public Float getObject()
|
||||
{
|
||||
return selector.getFloat();
|
||||
}
|
||||
@ -54,7 +54,7 @@ public class MetricSelectorUtils
|
||||
}
|
||||
|
||||
@Override
|
||||
public Double get()
|
||||
public Double getObject()
|
||||
{
|
||||
return selector.getDouble();
|
||||
}
|
||||
|
@ -41,7 +41,7 @@ public class TestObjectColumnSelector<T> implements ObjectColumnSelector
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
return objects[index];
|
||||
}
|
||||
|
@ -85,7 +85,7 @@ public class TestColumnSelectorFactory implements ColumnSelectorFactory
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
return row.get().getRaw(columnName);
|
||||
}
|
||||
|
@ -156,7 +156,7 @@ public class ExpressionObjectSelectorTest
|
||||
}
|
||||
|
||||
@Override
|
||||
public T get()
|
||||
public T getObject()
|
||||
{
|
||||
return supplier.get();
|
||||
}
|
||||
|
@ -95,16 +95,16 @@ public class ExpressionVirtualColumnTest
|
||||
final ObjectColumnSelector selector = XPLUSY.makeObjectColumnSelector("expr", COLUMN_SELECTOR_FACTORY);
|
||||
|
||||
COLUMN_SELECTOR_FACTORY.setRow(ROW0);
|
||||
Assert.assertEquals(null, selector.get());
|
||||
Assert.assertEquals(null, selector.getObject());
|
||||
|
||||
COLUMN_SELECTOR_FACTORY.setRow(ROW1);
|
||||
Assert.assertEquals(4.0d, selector.get());
|
||||
Assert.assertEquals(4.0d, selector.getObject());
|
||||
|
||||
COLUMN_SELECTOR_FACTORY.setRow(ROW2);
|
||||
Assert.assertEquals(5.1d, selector.get());
|
||||
Assert.assertEquals(5.1d, selector.getObject());
|
||||
|
||||
COLUMN_SELECTOR_FACTORY.setRow(ROW3);
|
||||
Assert.assertEquals(5L, selector.get());
|
||||
Assert.assertEquals(5L, selector.getObject());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -103,7 +103,7 @@ public class VirtualColumnsTest
|
||||
final FloatColumnSelector floatSelector = virtualColumns.makeFloatColumnSelector("expr", null);
|
||||
final LongColumnSelector longSelector = virtualColumns.makeLongColumnSelector("expr", null);
|
||||
|
||||
Assert.assertEquals(1L, objectSelector.get());
|
||||
Assert.assertEquals(1L, objectSelector.getObject());
|
||||
Assert.assertEquals("1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertEquals("0.5", extractionDimensionSelector.lookupName(extractionDimensionSelector.getRow().get(0)));
|
||||
Assert.assertEquals(1.0f, floatSelector.getFloat(), 0.0f);
|
||||
@ -122,7 +122,7 @@ public class VirtualColumnsTest
|
||||
final FloatColumnSelector floatSelector = virtualColumns.makeFloatColumnSelector("foo.5", null);
|
||||
final LongColumnSelector longSelector = virtualColumns.makeLongColumnSelector("foo.5", null);
|
||||
|
||||
Assert.assertEquals(5L, objectSelector.get());
|
||||
Assert.assertEquals(5L, objectSelector.getObject());
|
||||
Assert.assertEquals("5", dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertEquals(5.0f, floatSelector.getFloat(), 0.0f);
|
||||
Assert.assertEquals(5L, longSelector.getLong());
|
||||
@ -140,7 +140,7 @@ public class VirtualColumnsTest
|
||||
final FloatColumnSelector floatSelector = virtualColumns.makeFloatColumnSelector("foo", null);
|
||||
final LongColumnSelector longSelector = virtualColumns.makeLongColumnSelector("foo", null);
|
||||
|
||||
Assert.assertEquals(-1L, objectSelector.get());
|
||||
Assert.assertEquals(-1L, objectSelector.getObject());
|
||||
Assert.assertEquals("-1", dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertEquals(-1.0f, floatSelector.getFloat(), 0.0f);
|
||||
Assert.assertEquals(-1L, longSelector.getLong());
|
||||
@ -319,7 +319,7 @@ public class VirtualColumnsTest
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
return selector.getLong();
|
||||
}
|
||||
|
@ -149,7 +149,7 @@ public class IngestSegmentFirehose implements Firehose
|
||||
for (Map.Entry<String, ObjectColumnSelector> metSelector : metSelectors.entrySet()) {
|
||||
final String metric = metSelector.getKey();
|
||||
final ObjectColumnSelector selector = metSelector.getValue();
|
||||
theEvent.put(metric, selector.get());
|
||||
theEvent.put(metric, selector.getObject());
|
||||
}
|
||||
cursor.advance();
|
||||
return new MapBasedInputRow(timestamp, dims, theEvent);
|
||||
|
@ -288,7 +288,7 @@ public class DumpSegment extends GuiceRunnable
|
||||
|
||||
for (int i = 0; i < columnNames.size(); i++) {
|
||||
final String columnName = columnNames.get(i);
|
||||
final Object value = selectors.get(i).get();
|
||||
final Object value = selectors.get(i).getObject();
|
||||
|
||||
if (timeISO8601 && columnNames.get(i).equals(Column.TIME_COLUMN_NAME)) {
|
||||
row.put(columnName, new DateTime(value, DateTimeZone.UTC).toString());
|
||||
@ -525,7 +525,7 @@ public class DumpSegment extends GuiceRunnable
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> get()
|
||||
public List<String> getObject()
|
||||
{
|
||||
final IndexedInts row = dimensionSelector.getRow();
|
||||
if (row.size() == 0) {
|
||||
@ -549,7 +549,7 @@ public class DumpSegment extends GuiceRunnable
|
||||
}
|
||||
|
||||
@Override
|
||||
public String get()
|
||||
public String getObject()
|
||||
{
|
||||
final IndexedInts row = dimensionSelector.getRow();
|
||||
return row.size() == 0 ? null : dimensionSelector.lookupName(row.get(0));
|
||||
@ -572,7 +572,7 @@ public class DumpSegment extends GuiceRunnable
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
public Object getObject()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user