Fix double-checked locking in predicate suppliers in BoundDimFi… (#8974)

* Fix double-checked locking in predicate suppliers in BoundDimFilter

* Fix double-checked locking in predicate suppliers in BoundDimFilter

* 1. Use Suppliers.memoize() to initialize and publish singleton.

2. Fix coding style.

* Fix coding style

* Fix double-checked locking bug for predicate suppliers in InDimFilter
This commit is contained in:
Q 2019-12-04 09:01:52 -08:00 committed by Roman Leventov
parent d0a6fe7f12
commit 391646123e
2 changed files with 306 additions and 429 deletions

View File

@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.collect.BoundType;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Range;
@ -74,8 +75,7 @@ public class BoundDimFilter implements DimFilter
@Deprecated @JsonProperty("alphaNumeric") @Nullable Boolean alphaNumeric,
@JsonProperty("extractionFn") @Nullable ExtractionFn extractionFn,
@JsonProperty("ordering") @Nullable StringComparator ordering,
@JsonProperty("filterTuning") @Nullable FilterTuning filterTuning
)
@JsonProperty("filterTuning") @Nullable FilterTuning filterTuning)
{
this.dimension = Preconditions.checkNotNull(dimension, "dimension can not be null");
Preconditions.checkState((lower != null) || (upper != null), "lower and upper can not be null at the same time");
@ -84,8 +84,10 @@ public class BoundDimFilter implements DimFilter
this.lowerStrict = (lowerStrict == null) ? false : lowerStrict;
this.upperStrict = (upperStrict == null) ? false : upperStrict;
// For backwards compatibility, we retain the 'alphaNumeric' property. It will be used if the new 'ordering'
// property is missing. If both 'ordering' and 'alphaNumeric' are present, make sure they are consistent.
// For backwards compatibility, we retain the 'alphaNumeric' property. It
// will be used if the new 'ordering'
// property is missing. If both 'ordering' and 'alphaNumeric' are present,
// make sure they are consistent.
if (ordering == null) {
if (alphaNumeric == null || !alphaNumeric) {
this.ordering = StringComparators.LEXICOGRAPHIC;
@ -98,8 +100,7 @@ public class BoundDimFilter implements DimFilter
boolean orderingIsAlphanumeric = this.ordering.equals(StringComparators.ALPHANUMERIC);
Preconditions.checkState(
alphaNumeric == orderingIsAlphanumeric,
"mismatch between alphanumeric and ordering property"
);
"mismatch between alphanumeric and ordering property");
}
}
this.extractionFn = extractionFn;
@ -118,8 +119,7 @@ public class BoundDimFilter implements DimFilter
@Nullable Boolean upperStrict,
@Nullable Boolean alphaNumeric,
@Nullable ExtractionFn extractionFn,
@Nullable StringComparator ordering
)
@Nullable StringComparator ordering)
{
this(dimension, lower, upper, lowerStrict, upperStrict, alphaNumeric, extractionFn, ordering, null);
}
@ -224,26 +224,25 @@ public class BoundDimFilter implements DimFilter
ByteBuffer boundCacheBuffer = ByteBuffer.allocate(
9
+ dimensionBytes.length
+ upperBytes.length
+ lowerBytes.length
+ extractionFnBytes.length
+ orderingBytes.length
);
+ dimensionBytes.length
+ upperBytes.length
+ lowerBytes.length
+ extractionFnBytes.length
+ orderingBytes.length);
boundCacheBuffer.put(DimFilterUtils.BOUND_CACHE_ID)
.put(boundType)
.put(upperStrictByte)
.put(lowerStrictByte)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(dimensionBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(upperBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(lowerBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(extractionFnBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(orderingBytes);
.put(boundType)
.put(upperStrictByte)
.put(lowerStrictByte)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(dimensionBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(upperBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(lowerBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(extractionFnBytes)
.put(DimFilterUtils.STRING_SEPARATOR)
.put(orderingBytes);
return boundCacheBuffer.array();
}
@ -263,8 +262,8 @@ public class BoundDimFilter implements DimFilter
public RangeSet<String> getDimensionRangeSet(String dimension)
{
if (!(Objects.equals(getDimension(), dimension)
&& getExtractionFn() == null
&& ordering.equals(StringComparators.LEXICOGRAPHIC))) {
&& getExtractionFn() == null
&& ordering.equals(StringComparators.LEXICOGRAPHIC))) {
return null;
}
@ -276,8 +275,7 @@ public class BoundDimFilter implements DimFilter
range = isLowerStrict() ? Range.greaterThan(getLower()) : Range.atLeast(getLower());
} else {
range = Range.range(getLower(), isLowerStrict() ? BoundType.OPEN : BoundType.CLOSED,
getUpper(), isUpperStrict() ? BoundType.OPEN : BoundType.CLOSED
);
getUpper(), isUpperStrict() ? BoundType.OPEN : BoundType.CLOSED);
}
retSet.add(range);
return retSet;
@ -300,13 +298,13 @@ public class BoundDimFilter implements DimFilter
}
BoundDimFilter that = (BoundDimFilter) o;
return lowerStrict == that.lowerStrict &&
upperStrict == that.upperStrict &&
dimension.equals(that.dimension) &&
Objects.equals(upper, that.upper) &&
Objects.equals(lower, that.lower) &&
Objects.equals(extractionFn, that.extractionFn) &&
Objects.equals(ordering, that.ordering) &&
Objects.equals(filterTuning, that.filterTuning);
upperStrict == that.upperStrict &&
dimension.equals(that.dimension) &&
Objects.equals(upper, that.upper) &&
Objects.equals(lower, that.lower) &&
Objects.equals(extractionFn, that.extractionFn) &&
Objects.equals(ordering, that.ordering) &&
Objects.equals(filterTuning, that.filterTuning);
}
@Override
@ -320,8 +318,7 @@ public class BoundDimFilter implements DimFilter
upperStrict,
extractionFn,
ordering,
filterTuning
);
filterTuning);
}
@Override
@ -356,116 +353,97 @@ public class BoundDimFilter implements DimFilter
return builder.appendFilterTuning(filterTuning).build();
}
private Supplier<DruidLongPredicate> makeLongPredicateSupplier()
private DruidLongPredicate createLongPredicate()
{
class BoundLongPredicateSupplier implements Supplier<DruidLongPredicate>
{
private final Object initLock = new Object();
private DruidLongPredicate predicate;
boolean hasLowerLongBound;
boolean hasUpperLongBound;
long lowerLongBound;
long upperLongBound;
boolean matchesNothing = false;
@Override
public DruidLongPredicate get()
{
initPredicate();
return predicate;
}
private void initPredicate()
{
if (predicate != null) {
return;
}
synchronized (initLock) {
if (predicate != null) {
return;
}
boolean hasLowerLongBound;
boolean hasUpperLongBound;
long lowerLongBound;
long upperLongBound;
boolean matchesNothing = false;
if (hasLowerBound()) {
final Long lowerLong = GuavaUtils.tryParseLong(lower);
if (lowerLong == null) {
BigDecimal lowerBigDecimal = getBigDecimalLowerBoundFromFloatString(lower);
if (lowerBigDecimal == null) {
// Unparseable values fall before all actual numbers, so all numbers will match the lower bound.
hasLowerLongBound = false;
lowerLongBound = 0L;
} else {
try {
lowerLongBound = lowerBigDecimal.longValueExact();
hasLowerLongBound = true;
}
catch (ArithmeticException ae) { // the BigDecimal can't be contained in a long
hasLowerLongBound = false;
lowerLongBound = 0L;
if (lowerBigDecimal.compareTo(BigDecimal.ZERO) > 0) {
// positive lower bound, > all longs, will match nothing
matchesNothing = true;
}
}
}
} else {
hasLowerLongBound = true;
lowerLongBound = lowerLong;
}
} else {
if (hasLowerBound()) {
final Long lowerLong = GuavaUtils.tryParseLong(lower);
if (lowerLong == null) {
BigDecimal lowerBigDecimal = getBigDecimalLowerBoundFromFloatString(lower);
if (lowerBigDecimal == null) {
// Unparseable values fall before all actual numbers, so all numbers
// will match the lower bound.
hasLowerLongBound = false;
lowerLongBound = 0L;
} else {
try {
lowerLongBound = lowerBigDecimal.longValueExact();
hasLowerLongBound = true;
}
catch (ArithmeticException ae) { // the BigDecimal can't be contained in a long
hasLowerLongBound = false;
lowerLongBound = 0L;
}
if (hasUpperBound()) {
Long upperLong = GuavaUtils.tryParseLong(upper);
if (upperLong == null) {
BigDecimal upperBigDecimal = getBigDecimalUpperBoundFromFloatString(upper);
if (upperBigDecimal == null) {
// Unparseable values fall before all actual numbers, so no numbers can match the upper bound.
matchesNothing = true;
hasUpperLongBound = false;
upperLongBound = 0L;
} else {
try {
upperLongBound = upperBigDecimal.longValueExact();
hasUpperLongBound = true;
}
catch (ArithmeticException ae) { // the BigDecimal can't be contained in a long
hasUpperLongBound = false;
upperLongBound = 0L;
if (upperBigDecimal.compareTo(BigDecimal.ZERO) < 0) {
// negative upper bound, < all longs, will match nothing
matchesNothing = true;
}
}
}
} else {
hasUpperLongBound = true;
upperLongBound = upperLong;
if (lowerBigDecimal.compareTo(BigDecimal.ZERO) > 0) {
// positive lower bound, > all longs, will match nothing
matchesNothing = true;
}
} else {
hasUpperLongBound = false;
upperLongBound = 0L;
}
if (matchesNothing) {
predicate = DruidLongPredicate.ALWAYS_FALSE;
} else {
predicate = makeLongPredicateFromBounds(
hasLowerLongBound,
hasUpperLongBound,
lowerStrict,
upperStrict,
lowerLongBound,
upperLongBound
);
}
}
} else {
hasLowerLongBound = true;
lowerLongBound = lowerLong;
}
} else {
hasLowerLongBound = false;
lowerLongBound = 0L;
}
return new BoundLongPredicateSupplier();
if (hasUpperBound()) {
Long upperLong = GuavaUtils.tryParseLong(upper);
if (upperLong == null) {
BigDecimal upperBigDecimal = getBigDecimalUpperBoundFromFloatString(upper);
if (upperBigDecimal == null) {
// Unparseable values fall before all actual numbers, so no numbers
// can match the upper bound.
matchesNothing = true;
hasUpperLongBound = false;
upperLongBound = 0L;
} else {
try {
upperLongBound = upperBigDecimal.longValueExact();
hasUpperLongBound = true;
}
catch (ArithmeticException ae) { // the BigDecimal can't be
// contained in a long
hasUpperLongBound = false;
upperLongBound = 0L;
if (upperBigDecimal.compareTo(BigDecimal.ZERO) < 0) {
// negative upper bound, < all longs, will match nothing
matchesNothing = true;
}
}
}
} else {
hasUpperLongBound = true;
upperLongBound = upperLong;
}
} else {
hasUpperLongBound = false;
upperLongBound = 0L;
}
if (matchesNothing) {
return DruidLongPredicate.ALWAYS_FALSE;
} else {
return makeLongPredicateFromBounds(
hasLowerLongBound,
hasUpperLongBound,
lowerStrict,
upperStrict,
lowerLongBound,
upperLongBound);
}
}
private Supplier<DruidLongPredicate> makeLongPredicateSupplier()
{
Supplier<DruidLongPredicate> longPredicate = () -> createLongPredicate();
return Suppliers.memoize(longPredicate);
}
@Nullable
@ -474,7 +452,7 @@ public class BoundDimFilter implements DimFilter
BigDecimal convertedBD;
try {
convertedBD = new BigDecimal(floatStr);
}
}
catch (NumberFormatException nfe) {
return null;
}
@ -492,7 +470,7 @@ public class BoundDimFilter implements DimFilter
BigDecimal convertedBD;
try {
convertedBD = new BigDecimal(floatStr);
}
}
catch (NumberFormatException nfe) {
return null;
}
@ -504,167 +482,127 @@ public class BoundDimFilter implements DimFilter
}
}
private DruidFloatPredicate createDruidFloatPredicate()
{
final boolean hasLowerFloatBound;
final boolean hasUpperFloatBound;
final float lowerFloatBound;
final float upperFloatBound;
boolean matchesNothing = false;
if (hasLowerBound()) {
final Float lowerFloat = Floats.tryParse(lower);
if (lowerFloat == null) {
// Unparseable values fall before all actual numbers, so all numbers
// will match the lower bound.
hasLowerFloatBound = false;
lowerFloatBound = 0L;
} else {
hasLowerFloatBound = true;
lowerFloatBound = lowerFloat;
}
} else {
hasLowerFloatBound = false;
lowerFloatBound = 0L;
}
if (hasUpperBound()) {
Float upperFloat = Floats.tryParse(upper);
if (upperFloat == null) {
// Unparseable values fall before all actual numbers, so no numbers
// can match the upper bound.
matchesNothing = true;
hasUpperFloatBound = false;
upperFloatBound = 0L;
} else {
hasUpperFloatBound = true;
upperFloatBound = upperFloat;
}
} else {
hasUpperFloatBound = false;
upperFloatBound = 0L;
}
if (matchesNothing) {
return DruidFloatPredicate.ALWAYS_FALSE;
} else {
return input -> {
final DruidDoublePredicate druidDoublePredicate = makeDoublePredicateFromBounds(
hasLowerFloatBound,
hasUpperFloatBound,
lowerStrict,
upperStrict,
(double) lowerFloatBound,
(double) upperFloatBound);
return druidDoublePredicate.applyDouble((double) input);
};
}
}
private Supplier<DruidFloatPredicate> makeFloatPredicateSupplier()
{
class BoundFloatPredicateSupplier implements Supplier<DruidFloatPredicate>
{
private final Object initLock = new Object();
private DruidFloatPredicate predicate;
Supplier<DruidFloatPredicate> floatPredicate = () -> createDruidFloatPredicate();
return Suppliers.memoize(floatPredicate);
}
@Override
public DruidFloatPredicate get()
{
initPredicate();
return predicate;
}
private void initPredicate()
{
if (predicate != null) {
return;
}
synchronized (initLock) {
if (predicate != null) {
return;
}
final boolean hasLowerFloatBound;
final boolean hasUpperFloatBound;
final float lowerFloatBound;
final float upperFloatBound;
boolean matchesNothing = false;
if (hasLowerBound()) {
final Float lowerFloat = Floats.tryParse(lower);
if (lowerFloat == null) {
// Unparseable values fall before all actual numbers, so all numbers will match the lower bound.
hasLowerFloatBound = false;
lowerFloatBound = 0L;
} else {
hasLowerFloatBound = true;
lowerFloatBound = lowerFloat;
}
} else {
hasLowerFloatBound = false;
lowerFloatBound = 0L;
}
if (hasUpperBound()) {
Float upperFloat = Floats.tryParse(upper);
if (upperFloat == null) {
// Unparseable values fall before all actual numbers, so no numbers can match the upper bound.
matchesNothing = true;
hasUpperFloatBound = false;
upperFloatBound = 0L;
} else {
hasUpperFloatBound = true;
upperFloatBound = upperFloat;
}
} else {
hasUpperFloatBound = false;
upperFloatBound = 0L;
}
if (matchesNothing) {
predicate = DruidFloatPredicate.ALWAYS_FALSE;
} else {
predicate = input -> {
final DruidDoublePredicate druidDoublePredicate = makeDoublePredicateFromBounds(
hasLowerFloatBound,
hasUpperFloatBound,
lowerStrict,
upperStrict,
(double) lowerFloatBound,
(double) upperFloatBound
);
return druidDoublePredicate.applyDouble((double) input);
};
}
}
private DruidDoublePredicate createDruidDoublePredicate()
{
final boolean hasLowerBound;
final boolean hasUpperBound;
final double lowerDoubleBound;
final double upperDoubleBound;
boolean matchesNothing = false;
if (hasLowerBound()) {
final Double lowerDouble = Doubles.tryParse(lower);
if (lowerDouble == null) {
// Unparseable values fall before all actual numbers, so all numbers
// will match the lower bound.
hasLowerBound = false;
lowerDoubleBound = 0L;
} else {
hasLowerBound = true;
lowerDoubleBound = lowerDouble;
}
} else {
hasLowerBound = false;
lowerDoubleBound = 0L;
}
if (hasUpperBound()) {
Double upperDouble = Doubles.tryParse(upper);
if (upperDouble == null) {
// Unparseable values fall before all actual numbers, so no numbers can
// match the upper bound.
matchesNothing = true;
hasUpperBound = false;
upperDoubleBound = 0L;
} else {
hasUpperBound = true;
upperDoubleBound = upperDouble;
}
} else {
hasUpperBound = false;
upperDoubleBound = 0L;
}
if (matchesNothing) {
return DruidDoublePredicate.ALWAYS_FALSE;
} else {
return makeDoublePredicateFromBounds(
hasLowerBound,
hasUpperBound,
lowerStrict,
upperStrict,
lowerDoubleBound,
upperDoubleBound);
}
return new BoundFloatPredicateSupplier();
}
private Supplier<DruidDoublePredicate> makeDoublePredicateSupplier()
{
class BoundDoublePredicateSupplier implements Supplier<DruidDoublePredicate>
{
private final Object initLock = new Object();
private DruidDoublePredicate predicate;
@Override
public DruidDoublePredicate get()
{
initPredicate();
return predicate;
}
private void initPredicate()
{
if (predicate != null) {
return;
}
synchronized (initLock) {
if (predicate != null) {
return;
}
final boolean hasLowerBound;
final boolean hasUpperBound;
final double lowerDoubleBound;
final double upperDoubleBound;
boolean matchesNothing = false;
if (hasLowerBound()) {
final Double lowerDouble = Doubles.tryParse(lower);
if (lowerDouble == null) {
// Unparseable values fall before all actual numbers, so all numbers will match the lower bound.
hasLowerBound = false;
lowerDoubleBound = 0L;
} else {
hasLowerBound = true;
lowerDoubleBound = lowerDouble;
}
} else {
hasLowerBound = false;
lowerDoubleBound = 0L;
}
if (hasUpperBound()) {
Double upperDouble = Doubles.tryParse(upper);
if (upperDouble == null) {
// Unparseable values fall before all actual numbers, so no numbers can match the upper bound.
matchesNothing = true;
hasUpperBound = false;
upperDoubleBound = 0L;
} else {
hasUpperBound = true;
upperDoubleBound = upperDouble;
}
} else {
hasUpperBound = false;
upperDoubleBound = 0L;
}
if (matchesNothing) {
predicate = DruidDoublePredicate.ALWAYS_FALSE;
} else {
predicate = makeDoublePredicateFromBounds(
hasLowerBound,
hasUpperBound,
lowerStrict,
upperStrict,
lowerDoubleBound,
upperDoubleBound
);
}
}
}
}
return new BoundDoublePredicateSupplier();
Supplier<DruidDoublePredicate> doublePredicate = () -> createDruidDoublePredicate();
return Suppliers.memoize(doublePredicate);
}
private static DruidLongPredicate makeLongPredicateFromBounds(
@ -673,8 +611,7 @@ public class BoundDimFilter implements DimFilter
final boolean lowerStrict,
final boolean upperStrict,
final long lowerLongBound,
final long upperLongBound
)
final long upperLongBound)
{
if (hasLowerLongBound && hasUpperLongBound) {
if (upperStrict && lowerStrict) {
@ -777,8 +714,7 @@ public class BoundDimFilter implements DimFilter
final boolean lowerStrict,
final boolean upperStrict,
final double lowerDoubleBound,
final double upperDoubleBound
)
final double upperDoubleBound)
{
if (hasLowerDoubleBound && hasUpperDoubleBound) {
if (upperStrict && lowerStrict) {

View File

@ -26,6 +26,7 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Range;
@ -271,152 +272,92 @@ public class InDimFilter implements DimFilter
{
return Objects.hash(values, dimension, extractionFn, filterTuning);
}
private DruidLongPredicate createLongPredicate()
{
LongArrayList longs = new LongArrayList(values.size());
for (String value : values) {
final Long longValue = DimensionHandlerUtils.getExactLongFromDecimalString(value);
if (longValue != null) {
longs.add(longValue);
}
}
if (longs.size() > NUMERIC_HASHING_THRESHOLD) {
final LongOpenHashSet longHashSet = new LongOpenHashSet(longs);
return input -> longHashSet.contains(input);
} else {
final long[] longArray = longs.toLongArray();
Arrays.sort(longArray);
return input -> Arrays.binarySearch(longArray, input) >= 0;
}
}
// As the set of filtered values can be large, parsing them as longs should be done only if needed, and only once.
// Pass in a common long predicate supplier to all filters created by .toFilter(), so that
// we only compute the long hashset/array once per query.
// This supplier must be thread-safe, since this DimFilter will be accessed in the query runners.
private Supplier<DruidLongPredicate> getLongPredicateSupplier()
{
return new Supplier<DruidLongPredicate>()
{
private final Object initLock = new Object();
private DruidLongPredicate predicate;
private void initLongValues()
{
if (predicate != null) {
return;
}
synchronized (initLock) {
if (predicate != null) {
return;
}
LongArrayList longs = new LongArrayList(values.size());
for (String value : values) {
final Long longValue = DimensionHandlerUtils.getExactLongFromDecimalString(value);
if (longValue != null) {
longs.add(longValue);
}
}
if (longs.size() > NUMERIC_HASHING_THRESHOLD) {
final LongOpenHashSet longHashSet = new LongOpenHashSet(longs);
predicate = input -> longHashSet.contains(input);
} else {
final long[] longArray = longs.toLongArray();
Arrays.sort(longArray);
predicate = input -> Arrays.binarySearch(longArray, input) >= 0;
}
}
}
@Override
public DruidLongPredicate get()
{
initLongValues();
return predicate;
}
};
Supplier<DruidLongPredicate> longPredicate = () -> createLongPredicate();
return Suppliers.memoize(longPredicate);
}
private DruidFloatPredicate createFloatPredicate()
{
IntArrayList floatBits = new IntArrayList(values.size());
for (String value : values) {
Float floatValue = Floats.tryParse(value);
if (floatValue != null) {
floatBits.add(Float.floatToIntBits(floatValue));
}
}
if (floatBits.size() > NUMERIC_HASHING_THRESHOLD) {
final IntOpenHashSet floatBitsHashSet = new IntOpenHashSet(floatBits);
return input -> floatBitsHashSet.contains(Float.floatToIntBits(input));
} else {
final int[] floatBitsArray = floatBits.toIntArray();
Arrays.sort(floatBitsArray);
return input -> Arrays.binarySearch(floatBitsArray, Float.floatToIntBits(input)) >= 0;
}
}
private Supplier<DruidFloatPredicate> getFloatPredicateSupplier()
{
return new Supplier<DruidFloatPredicate>()
{
private final Object initLock = new Object();
private DruidFloatPredicate predicate;
private void initFloatValues()
{
if (predicate != null) {
return;
}
synchronized (initLock) {
if (predicate != null) {
return;
}
IntArrayList floatBits = new IntArrayList(values.size());
for (String value : values) {
Float floatValue = Floats.tryParse(value);
if (floatValue != null) {
floatBits.add(Float.floatToIntBits(floatValue));
}
}
if (floatBits.size() > NUMERIC_HASHING_THRESHOLD) {
final IntOpenHashSet floatBitsHashSet = new IntOpenHashSet(floatBits);
predicate = input -> floatBitsHashSet.contains(Float.floatToIntBits(input));
} else {
final int[] floatBitsArray = floatBits.toIntArray();
Arrays.sort(floatBitsArray);
predicate = input -> Arrays.binarySearch(floatBitsArray, Float.floatToIntBits(input)) >= 0;
}
}
Supplier<DruidFloatPredicate> floatPredicate = () -> createFloatPredicate();
return Suppliers.memoize(floatPredicate);
}
private DruidDoublePredicate createDoublePredicate()
{
LongArrayList doubleBits = new LongArrayList(values.size());
for (String value : values) {
Double doubleValue = Doubles.tryParse(value);
if (doubleValue != null) {
doubleBits.add(Double.doubleToLongBits((doubleValue)));
}
}
@Override
public DruidFloatPredicate get()
{
initFloatValues();
return predicate;
}
};
if (doubleBits.size() > NUMERIC_HASHING_THRESHOLD) {
final LongOpenHashSet doubleBitsHashSet = new LongOpenHashSet(doubleBits);
return input -> doubleBitsHashSet.contains(Double.doubleToLongBits(input));
} else {
final long[] doubleBitsArray = doubleBits.toLongArray();
Arrays.sort(doubleBitsArray);
return input -> Arrays.binarySearch(doubleBitsArray, Double.doubleToLongBits(input)) >= 0;
}
}
private Supplier<DruidDoublePredicate> getDoublePredicateSupplier()
{
return new Supplier<DruidDoublePredicate>()
{
private final Object initLock = new Object();
private DruidDoublePredicate predicate;
private void initDoubleValues()
{
if (predicate != null) {
return;
}
synchronized (initLock) {
if (predicate != null) {
return;
}
LongArrayList doubleBits = new LongArrayList(values.size());
for (String value : values) {
Double doubleValue = Doubles.tryParse(value);
if (doubleValue != null) {
doubleBits.add(Double.doubleToLongBits((doubleValue)));
}
}
if (doubleBits.size() > NUMERIC_HASHING_THRESHOLD) {
final LongOpenHashSet doubleBitsHashSet = new LongOpenHashSet(doubleBits);
predicate = input -> doubleBitsHashSet.contains(Double.doubleToLongBits(input));
} else {
final long[] doubleBitsArray = doubleBits.toLongArray();
Arrays.sort(doubleBitsArray);
predicate = input -> Arrays.binarySearch(doubleBitsArray, Double.doubleToLongBits(input)) >= 0;
}
}
}
@Override
public DruidDoublePredicate get()
{
initDoubleValues();
return predicate;
}
};
Supplier<DruidDoublePredicate> doublePredicate = () -> createDoublePredicate();
return Suppliers.memoize(doublePredicate);
}
}