mirror of https://github.com/apache/druid.git
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:
parent
d0a6fe7f12
commit
391646123e
|
@ -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);
|
||||
}
|
||||
|
@ -228,8 +228,7 @@ public class BoundDimFilter implements DimFilter
|
|||
+ upperBytes.length
|
||||
+ lowerBytes.length
|
||||
+ extractionFnBytes.length
|
||||
+ orderingBytes.length
|
||||
);
|
||||
+ orderingBytes.length);
|
||||
boundCacheBuffer.put(DimFilterUtils.BOUND_CACHE_ID)
|
||||
.put(boundType)
|
||||
.put(upperStrictByte)
|
||||
|
@ -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;
|
||||
|
@ -320,8 +318,7 @@ public class BoundDimFilter implements DimFilter
|
|||
upperStrict,
|
||||
extractionFn,
|
||||
ordering,
|
||||
filterTuning
|
||||
);
|
||||
filterTuning);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -356,31 +353,8 @@ 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;
|
||||
|
||||
@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;
|
||||
|
@ -392,7 +366,8 @@ public class BoundDimFilter implements DimFilter
|
|||
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.
|
||||
// Unparseable values fall before all actual numbers, so all numbers
|
||||
// will match the lower bound.
|
||||
hasLowerLongBound = false;
|
||||
lowerLongBound = 0L;
|
||||
} else {
|
||||
|
@ -423,7 +398,8 @@ public class BoundDimFilter implements DimFilter
|
|||
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.
|
||||
// Unparseable values fall before all actual numbers, so no numbers
|
||||
// can match the upper bound.
|
||||
matchesNothing = true;
|
||||
hasUpperLongBound = false;
|
||||
upperLongBound = 0L;
|
||||
|
@ -432,7 +408,8 @@ public class BoundDimFilter implements DimFilter
|
|||
upperLongBound = upperBigDecimal.longValueExact();
|
||||
hasUpperLongBound = true;
|
||||
}
|
||||
catch (ArithmeticException ae) { // the BigDecimal can't be contained in a long
|
||||
catch (ArithmeticException ae) { // the BigDecimal can't be
|
||||
// contained in a long
|
||||
hasUpperLongBound = false;
|
||||
upperLongBound = 0L;
|
||||
if (upperBigDecimal.compareTo(BigDecimal.ZERO) < 0) {
|
||||
|
@ -451,21 +428,22 @@ public class BoundDimFilter implements DimFilter
|
|||
}
|
||||
|
||||
if (matchesNothing) {
|
||||
predicate = DruidLongPredicate.ALWAYS_FALSE;
|
||||
return DruidLongPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
predicate = makeLongPredicateFromBounds(
|
||||
return makeLongPredicateFromBounds(
|
||||
hasLowerLongBound,
|
||||
hasUpperLongBound,
|
||||
lowerStrict,
|
||||
upperStrict,
|
||||
lowerLongBound,
|
||||
upperLongBound
|
||||
);
|
||||
upperLongBound);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return new BoundLongPredicateSupplier();
|
||||
|
||||
private Supplier<DruidLongPredicate> makeLongPredicateSupplier()
|
||||
{
|
||||
Supplier<DruidLongPredicate> longPredicate = () -> createLongPredicate();
|
||||
return Suppliers.memoize(longPredicate);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -504,31 +482,8 @@ public class BoundDimFilter implements DimFilter
|
|||
}
|
||||
}
|
||||
|
||||
private Supplier<DruidFloatPredicate> makeFloatPredicateSupplier()
|
||||
private DruidFloatPredicate createDruidFloatPredicate()
|
||||
{
|
||||
class BoundFloatPredicateSupplier implements Supplier<DruidFloatPredicate>
|
||||
{
|
||||
private final Object initLock = new Object();
|
||||
private DruidFloatPredicate predicate;
|
||||
|
||||
@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;
|
||||
|
@ -538,7 +493,8 @@ public class BoundDimFilter implements DimFilter
|
|||
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.
|
||||
// Unparseable values fall before all actual numbers, so all numbers
|
||||
// will match the lower bound.
|
||||
hasLowerFloatBound = false;
|
||||
lowerFloatBound = 0L;
|
||||
} else {
|
||||
|
@ -553,7 +509,8 @@ public class BoundDimFilter implements DimFilter
|
|||
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.
|
||||
// Unparseable values fall before all actual numbers, so no numbers
|
||||
// can match the upper bound.
|
||||
matchesNothing = true;
|
||||
hasUpperFloatBound = false;
|
||||
upperFloatBound = 0L;
|
||||
|
@ -567,51 +524,29 @@ public class BoundDimFilter implements DimFilter
|
|||
}
|
||||
|
||||
if (matchesNothing) {
|
||||
predicate = DruidFloatPredicate.ALWAYS_FALSE;
|
||||
return DruidFloatPredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
predicate = input -> {
|
||||
return input -> {
|
||||
final DruidDoublePredicate druidDoublePredicate = makeDoublePredicateFromBounds(
|
||||
hasLowerFloatBound,
|
||||
hasUpperFloatBound,
|
||||
lowerStrict,
|
||||
upperStrict,
|
||||
(double) lowerFloatBound,
|
||||
(double) upperFloatBound
|
||||
);
|
||||
(double) upperFloatBound);
|
||||
return druidDoublePredicate.applyDouble((double) input);
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return new BoundFloatPredicateSupplier();
|
||||
}
|
||||
|
||||
private Supplier<DruidDoublePredicate> makeDoublePredicateSupplier()
|
||||
private Supplier<DruidFloatPredicate> makeFloatPredicateSupplier()
|
||||
{
|
||||
class BoundDoublePredicateSupplier implements Supplier<DruidDoublePredicate>
|
||||
{
|
||||
private final Object initLock = new Object();
|
||||
private DruidDoublePredicate predicate;
|
||||
|
||||
@Override
|
||||
public DruidDoublePredicate get()
|
||||
{
|
||||
initPredicate();
|
||||
return predicate;
|
||||
Supplier<DruidFloatPredicate> floatPredicate = () -> createDruidFloatPredicate();
|
||||
return Suppliers.memoize(floatPredicate);
|
||||
}
|
||||
|
||||
private void initPredicate()
|
||||
private DruidDoublePredicate createDruidDoublePredicate()
|
||||
{
|
||||
if (predicate != null) {
|
||||
return;
|
||||
}
|
||||
|
||||
synchronized (initLock) {
|
||||
if (predicate != null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final boolean hasLowerBound;
|
||||
final boolean hasUpperBound;
|
||||
final double lowerDoubleBound;
|
||||
|
@ -621,7 +556,8 @@ public class BoundDimFilter implements DimFilter
|
|||
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.
|
||||
// Unparseable values fall before all actual numbers, so all numbers
|
||||
// will match the lower bound.
|
||||
hasLowerBound = false;
|
||||
lowerDoubleBound = 0L;
|
||||
} else {
|
||||
|
@ -636,7 +572,8 @@ public class BoundDimFilter implements DimFilter
|
|||
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.
|
||||
// Unparseable values fall before all actual numbers, so no numbers can
|
||||
// match the upper bound.
|
||||
matchesNothing = true;
|
||||
hasUpperBound = false;
|
||||
upperDoubleBound = 0L;
|
||||
|
@ -650,21 +587,22 @@ public class BoundDimFilter implements DimFilter
|
|||
}
|
||||
|
||||
if (matchesNothing) {
|
||||
predicate = DruidDoublePredicate.ALWAYS_FALSE;
|
||||
return DruidDoublePredicate.ALWAYS_FALSE;
|
||||
} else {
|
||||
predicate = makeDoublePredicateFromBounds(
|
||||
return makeDoublePredicateFromBounds(
|
||||
hasLowerBound,
|
||||
hasUpperBound,
|
||||
lowerStrict,
|
||||
upperStrict,
|
||||
lowerDoubleBound,
|
||||
upperDoubleBound
|
||||
);
|
||||
upperDoubleBound);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return new BoundDoublePredicateSupplier();
|
||||
|
||||
private Supplier<DruidDoublePredicate> makeDoublePredicateSupplier()
|
||||
{
|
||||
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) {
|
||||
|
|
|
@ -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;
|
||||
|
@ -272,29 +273,8 @@ public class InDimFilter implements DimFilter
|
|||
return Objects.hash(values, dimension, extractionFn, filterTuning);
|
||||
}
|
||||
|
||||
// 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()
|
||||
private DruidLongPredicate createLongPredicate()
|
||||
{
|
||||
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);
|
||||
|
@ -306,43 +286,27 @@ public class InDimFilter implements DimFilter
|
|||
if (longs.size() > NUMERIC_HASHING_THRESHOLD) {
|
||||
final LongOpenHashSet longHashSet = new LongOpenHashSet(longs);
|
||||
|
||||
predicate = input -> longHashSet.contains(input);
|
||||
return input -> longHashSet.contains(input);
|
||||
} else {
|
||||
final long[] longArray = longs.toLongArray();
|
||||
Arrays.sort(longArray);
|
||||
|
||||
predicate = input -> Arrays.binarySearch(longArray, input) >= 0;
|
||||
}
|
||||
return input -> Arrays.binarySearch(longArray, input) >= 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidLongPredicate get()
|
||||
// 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()
|
||||
{
|
||||
initLongValues();
|
||||
return predicate;
|
||||
}
|
||||
};
|
||||
Supplier<DruidLongPredicate> longPredicate = () -> createLongPredicate();
|
||||
return Suppliers.memoize(longPredicate);
|
||||
}
|
||||
|
||||
private Supplier<DruidFloatPredicate> getFloatPredicateSupplier()
|
||||
private DruidFloatPredicate createFloatPredicate()
|
||||
{
|
||||
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);
|
||||
|
@ -354,43 +318,23 @@ public class InDimFilter implements DimFilter
|
|||
if (floatBits.size() > NUMERIC_HASHING_THRESHOLD) {
|
||||
final IntOpenHashSet floatBitsHashSet = new IntOpenHashSet(floatBits);
|
||||
|
||||
predicate = input -> floatBitsHashSet.contains(Float.floatToIntBits(input));
|
||||
return input -> floatBitsHashSet.contains(Float.floatToIntBits(input));
|
||||
} else {
|
||||
final int[] floatBitsArray = floatBits.toIntArray();
|
||||
Arrays.sort(floatBitsArray);
|
||||
|
||||
predicate = input -> Arrays.binarySearch(floatBitsArray, Float.floatToIntBits(input)) >= 0;
|
||||
}
|
||||
return input -> Arrays.binarySearch(floatBitsArray, Float.floatToIntBits(input)) >= 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidFloatPredicate get()
|
||||
private Supplier<DruidFloatPredicate> getFloatPredicateSupplier()
|
||||
{
|
||||
initFloatValues();
|
||||
return predicate;
|
||||
}
|
||||
};
|
||||
Supplier<DruidFloatPredicate> floatPredicate = () -> createFloatPredicate();
|
||||
return Suppliers.memoize(floatPredicate);
|
||||
}
|
||||
|
||||
private Supplier<DruidDoublePredicate> getDoublePredicateSupplier()
|
||||
private DruidDoublePredicate createDoublePredicate()
|
||||
{
|
||||
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);
|
||||
|
@ -402,21 +346,18 @@ public class InDimFilter implements DimFilter
|
|||
if (doubleBits.size() > NUMERIC_HASHING_THRESHOLD) {
|
||||
final LongOpenHashSet doubleBitsHashSet = new LongOpenHashSet(doubleBits);
|
||||
|
||||
predicate = input -> doubleBitsHashSet.contains(Double.doubleToLongBits(input));
|
||||
return input -> doubleBitsHashSet.contains(Double.doubleToLongBits(input));
|
||||
} else {
|
||||
final long[] doubleBitsArray = doubleBits.toLongArray();
|
||||
Arrays.sort(doubleBitsArray);
|
||||
|
||||
predicate = input -> Arrays.binarySearch(doubleBitsArray, Double.doubleToLongBits(input)) >= 0;
|
||||
return input -> Arrays.binarySearch(doubleBitsArray, Double.doubleToLongBits(input)) >= 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public DruidDoublePredicate get()
|
||||
|
||||
private Supplier<DruidDoublePredicate> getDoublePredicateSupplier()
|
||||
{
|
||||
initDoubleValues();
|
||||
return predicate;
|
||||
}
|
||||
};
|
||||
Supplier<DruidDoublePredicate> doublePredicate = () -> createDoublePredicate();
|
||||
return Suppliers.memoize(doublePredicate);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue