Use Double.NEGATIVE_INFINITY and Double.POSITIVE_INFINITY (#4496)

* Use Double.NEGATIVE_INFINITY and Double.POSITIVE_INFINITY instead of Double.MIN_VALUE and Double.MAX_VALUE, same for Float

* Replace usages in comments

* Fix RTree

* Remove commented code

* Add tests
This commit is contained in:
Roman Leventov 2017-07-07 10:10:13 -05:00 committed by Gian Merlino
parent 2641fecef9
commit d168a4271e
17 changed files with 123 additions and 40 deletions

View File

@ -0,0 +1,41 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.jackson;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
public class JacksonExtremeDoubleValuesSerdeTest
{
@Test
public void testExtremeDoubleValuesSerde() throws IOException
{
ObjectMapper objectMapper = new ObjectMapper();
for (double value : new double[] {Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY}) {
String serialized = objectMapper.writeValueAsString(value);
Assert.assertEquals(new Double(value), objectMapper.readValue(serialized, Double.class));
}
String negativeInfinityString = objectMapper.writeValueAsString(Double.NaN);
Assert.assertTrue(objectMapper.readValue(negativeInfinityString, Double.class).isNaN());
}
}

View File

@ -151,9 +151,9 @@ public class Node
{ {
boolean retVal = false; boolean retVal = false;
float[] minCoords = new float[getNumDims()]; float[] minCoords = new float[getNumDims()];
Arrays.fill(minCoords, Float.MAX_VALUE); Arrays.fill(minCoords, Float.POSITIVE_INFINITY);
float[] maxCoords = new float[getNumDims()]; float[] maxCoords = new float[getNumDims()];
Arrays.fill(maxCoords, -Float.MAX_VALUE); Arrays.fill(maxCoords, Float.NEGATIVE_INFINITY);
for (Node child : getChildren()) { for (Node child : getChildren()) {
for (int i = 0; i < getNumDims(); i++) { for (int i = 0; i < getNumDims(); i++) {

View File

@ -128,8 +128,8 @@ public class RTree
{ {
float[] initMinCoords = new float[numDims]; float[] initMinCoords = new float[numDims];
float[] initMaxCoords = new float[numDims]; float[] initMaxCoords = new float[numDims];
Arrays.fill(initMinCoords, -Float.MAX_VALUE); Arrays.fill(initMinCoords, Float.NEGATIVE_INFINITY);
Arrays.fill(initMaxCoords, Float.MAX_VALUE); Arrays.fill(initMaxCoords, Float.POSITIVE_INFINITY);
return new Node(initMinCoords, initMaxCoords, isLeaf, bitmapFactory); return new Node(initMinCoords, initMaxCoords, isLeaf, bitmapFactory);
} }
@ -178,7 +178,7 @@ public class RTree
return node; return node;
} }
double minCost = Double.MAX_VALUE; double minCost = Double.POSITIVE_INFINITY;
Node optimal = node.getChildren().get(0); Node optimal = node.getChildren().get(0);
for (Node child : node.getChildren()) { for (Node child : node.getChildren()) {
double cost = RTreeUtils.getExpansionCost(child, point); double cost = RTreeUtils.getExpansionCost(child, point);

View File

@ -58,10 +58,10 @@ public class LinearGutmanSplitStrategy extends GutmanSplitStrategy
double bestNormalized = 0.0; double bestNormalized = 0.0;
for (int i = 0; i < numDims; i++) { for (int i = 0; i < numDims; i++) {
float minCoord = Float.MAX_VALUE; float minCoord = Float.POSITIVE_INFINITY;
float maxCoord = -Float.MAX_VALUE; float maxCoord = Float.NEGATIVE_INFINITY;
float highestLowSide = -Float.MAX_VALUE; float lowestHighside = Float.POSITIVE_INFINITY;
float lowestHighside = Float.MAX_VALUE; float highestLowSide = Float.NEGATIVE_INFINITY;
int highestLowSideIndex = 0; int highestLowSideIndex = 0;
int lowestHighSideIndex = 0; int lowestHighSideIndex = 0;

View File

@ -37,7 +37,7 @@ public class QuadraticGutmanSplitStrategy extends GutmanSplitStrategy
@Override @Override
public Node[] pickSeeds(List<Node> nodes) public Node[] pickSeeds(List<Node> nodes)
{ {
double highestCost = Double.MIN_VALUE; double highestCost = Double.NEGATIVE_INFINITY;
int[] highestCostIndices = new int[2]; int[] highestCostIndices = new int[2];
for (int i = 0; i < nodes.size() - 1; i++) { for (int i = 0; i < nodes.size() - 1; i++) {
@ -58,7 +58,7 @@ public class QuadraticGutmanSplitStrategy extends GutmanSplitStrategy
@Override @Override
public Node pickNext(List<Node> nodes, Node[] groups) public Node pickNext(List<Node> nodes, Node[] groups)
{ {
double highestCost = Double.MIN_VALUE; double highestCost = Double.NEGATIVE_INFINITY;
Node costlyNode = null; Node costlyNode = null;
int counter = 0; int counter = 0;
int index = -1; int index = -1;

View File

@ -71,10 +71,31 @@
<property name="illegalPattern" value="true"/> <property name="illegalPattern" value="true"/>
<property name="message" value="Use Comparators.naturalNullsFirst() instead of Ordering.natural().nullsFirst()"/> <property name="message" value="Use Comparators.naturalNullsFirst() instead of Ordering.natural().nullsFirst()"/>
</module> </module>
<module name="Regexp"> <module name="Regexp">
<property name="format" value="(Byte|Character|Short|Integer|Long|Float|Double)\.TYPE"/> <property name="format" value="(Byte|Character|Short|Integer|Long|Float|Double)\.TYPE"/>
<property name="illegalPattern" value="true"/> <property name="illegalPattern" value="true"/>
<property name="message" value="Use primitive.class instead. But check twice that you don't actually need BoxedPrimitive.class instead of BoxedPrimitive.TYPE"/> <property name="message" value="Use primitive.class instead. But check twice that you don't actually need BoxedPrimitive.class instead of BoxedPrimitive.TYPE"/>
</module> </module>
<module name="Regexp">
<property name="format" value="Float\.MAX_VALUE"/>
<property name="illegalPattern" value="true"/>
<property name="message" value="Use Float.POSITIVE_INFINITY"/>
</module>
<module name="Regexp">
<property name="format" value="Float\.MIN_VALUE"/>
<property name="illegalPattern" value="true"/>
<property name="message" value="Use Float.NEGATIVE_INFINITY"/>
</module>
<module name="Regexp">
<property name="format" value="Double\.MAX_VALUE"/>
<property name="illegalPattern" value="true"/>
<property name="message" value="Use Double.POSITIVE_INFINITY"/>
</module>
<module name="Regexp">
<property name="format" value="Double\.MIN_VALUE"/>
<property name="illegalPattern" value="true"/>
<property name="message" value="Use Double.NEGATIVE_INFINITY"/>
</module>
</module> </module>
</module> </module>

View File

@ -335,11 +335,11 @@ public class ApproximateHistogram
// or merge existing bins before inserting the new one // or merge existing bins before inserting the new one
int minPos = minDeltaIndex(); int minPos = minDeltaIndex();
float minDelta = minPos >= 0 ? positions[minPos + 1] - positions[minPos] : Float.MAX_VALUE; float minDelta = minPos >= 0 ? positions[minPos + 1] - positions[minPos] : Float.POSITIVE_INFINITY;
// determine the distance of new value to the nearest bins // determine the distance of new value to the nearest bins
final float deltaRight = insertAt < binCount ? positions[insertAt] - value : Float.MAX_VALUE; final float deltaRight = insertAt < binCount ? positions[insertAt] - value : Float.POSITIVE_INFINITY;
final float deltaLeft = insertAt > 0 ? value - positions[insertAt - 1] : Float.MAX_VALUE; final float deltaLeft = insertAt > 0 ? value - positions[insertAt - 1] : Float.POSITIVE_INFINITY;
boolean mergeValue = false; boolean mergeValue = false;
if (deltaRight < minDelta) { if (deltaRight < minDelta) {
@ -368,7 +368,7 @@ public class ApproximateHistogram
protected int minDeltaIndex() protected int minDeltaIndex()
{ {
// determine minimum distance between existing bins // determine minimum distance between existing bins
float minDelta = Float.MAX_VALUE; float minDelta = Float.POSITIVE_INFINITY;
int minPos = -1; int minPos = -1;
for (int i = 0; i < binCount - 1; ++i) { for (int i = 0; i < binCount - 1; ++i) {
float delta = (positions[i + 1] - positions[i]); float delta = (positions[i + 1] - positions[i]);
@ -886,9 +886,6 @@ public class ApproximateHistogram
while (i < numMerge) { while (i < numMerge) {
// find the smallest delta within the range used for bins // find the smallest delta within the range used for bins
// pick minimum delta by scanning array
//int currentIndex = minIndex(deltas, lastValidIndex);
// pick minimum delta index using min-heap // pick minimum delta index using min-heap
int currentIndex = heap[0]; int currentIndex = heap[0];
@ -908,17 +905,13 @@ public class ApproximateHistogram
final float mm0 = (m0 - m1) * w + m1; final float mm0 = (m0 - m1) * w + m1;
mergedPositions[currentIndex] = mm0; mergedPositions[currentIndex] = mm0;
//mergedPositions[nextIndex] = Float.MAX_VALUE; // for debugging
mergedBins[currentIndex] = sum | APPROX_FLAG_BIT; mergedBins[currentIndex] = sum | APPROX_FLAG_BIT;
//mergedBins[nextIndex] = -1; // for debugging
// update deltas and min-heap // update deltas and min-heap
if (nextIndex == lastValidIndex) { if (nextIndex == lastValidIndex) {
// merged bin is the last => remove the current bin delta from the heap // merged bin is the last => remove the current bin delta from the heap
heapSize = heapDelete(heap, reverseIndex, heapSize, reverseIndex[currentIndex], deltas); heapSize = heapDelete(heap, reverseIndex, heapSize, reverseIndex[currentIndex], deltas);
//deltas[currentIndex] = Float.MAX_VALUE; // for debugging
} else { } else {
// merged bin is not the last => remove the merged bin delta from the heap // merged bin is not the last => remove the merged bin delta from the heap
heapSize = heapDelete(heap, reverseIndex, heapSize, reverseIndex[nextIndex], deltas); heapSize = heapDelete(heap, reverseIndex, heapSize, reverseIndex[nextIndex], deltas);
@ -938,9 +931,6 @@ public class ApproximateHistogram
siftDown(heap, reverseIndex, reverseIndex[prevIndex], heapSize - 1, deltas); siftDown(heap, reverseIndex, reverseIndex[prevIndex], heapSize - 1, deltas);
} }
// mark the merged bin as invalid
// deltas[nextIndex] = Float.MAX_VALUE; // for debugging
// update last valid index if we merged the last bin // update last valid index if we merged the last bin
if (nextIndex == lastValidIndex) { if (nextIndex == lastValidIndex) {
lastValidIndex = currentIndex; lastValidIndex = currentIndex;
@ -1037,7 +1027,7 @@ public class ApproximateHistogram
private static int minIndex(float[] deltas, int lastValidIndex) private static int minIndex(float[] deltas, int lastValidIndex)
{ {
int minIndex = -1; int minIndex = -1;
float min = Float.MAX_VALUE; float min = Float.POSITIVE_INFINITY;
for (int k = 0; k < lastValidIndex; ++k) { for (int k = 0; k < lastValidIndex; ++k) {
float value = deltas[k]; float value = deltas[k];
if (value < min) { if (value < min) {

View File

@ -146,7 +146,7 @@ public abstract class HyperLogLogCollector implements Comparable<HyperLogLogColl
final double ratio = e / TWO_TO_THE_SIXTY_FOUR; final double ratio = e / TWO_TO_THE_SIXTY_FOUR;
if (ratio >= 1) { if (ratio >= 1) {
// handle very unlikely case that value is > 2^64 // handle very unlikely case that value is > 2^64
return Double.MAX_VALUE; return Double.POSITIVE_INFINITY;
} else { } else {
return -TWO_TO_THE_SIXTY_FOUR * Math.log(1 - ratio); return -TWO_TO_THE_SIXTY_FOUR * Math.log(1 - ratio);
} }

View File

@ -698,7 +698,7 @@ public class HyperLogLogCollectorTest
fillBuckets(collector, (byte) 0, (byte) 63); fillBuckets(collector, (byte) 0, (byte) 63);
collector.add(new byte[]{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}); collector.add(new byte[]{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0});
Assert.assertEquals(Double.MAX_VALUE, collector.estimateCardinality(), 1000); Assert.assertEquals(Double.POSITIVE_INFINITY, collector.estimateCardinality(), 1000);
} }
@Test @Test

View File

@ -86,7 +86,13 @@ public class DoubleMaxAggregatorFactory extends AggregatorFactory
private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory) private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory)
{ {
return AggregatorUtil.getFloatColumnSelector(metricFactory, macroTable, fieldName, expression, Float.MIN_VALUE); return AggregatorUtil.getFloatColumnSelector(
metricFactory,
macroTable,
fieldName,
expression,
Float.NEGATIVE_INFINITY
);
} }
@Override @Override

View File

@ -87,7 +87,13 @@ public class DoubleMinAggregatorFactory extends AggregatorFactory
private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory) private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory)
{ {
return AggregatorUtil.getFloatColumnSelector(metricFactory, macroTable, fieldName, expression, Float.MAX_VALUE); return AggregatorUtil.getFloatColumnSelector(
metricFactory,
macroTable,
fieldName,
expression,
Float.POSITIVE_INFINITY
);
} }
@Override @Override

View File

@ -42,8 +42,8 @@ public class Histogram
this.breaks = breaks; this.breaks = breaks;
this.bins = new long[this.breaks.length + 1]; this.bins = new long[this.breaks.length + 1];
this.count = 0; this.count = 0;
this.min = Float.MAX_VALUE; this.min = Float.POSITIVE_INFINITY;
this.max = Float.MIN_VALUE; this.max = Float.NEGATIVE_INFINITY;
} }
public Histogram(float[] breaks, long[] bins, float min, float max) { public Histogram(float[] breaks, long[] bins, float min, float max) {

View File

@ -50,8 +50,8 @@ public class HistogramBufferAggregator implements BufferAggregator
final long[] bins = new long[breaks.length + 1]; final long[] bins = new long[breaks.length + 1];
mutationBuffer.asLongBuffer().put(bins); mutationBuffer.asLongBuffer().put(bins);
mutationBuffer.putFloat(position + minOffset, Float.MAX_VALUE); mutationBuffer.putFloat(position + minOffset, Float.POSITIVE_INFINITY);
mutationBuffer.putFloat(position + maxOffset, Float.MIN_VALUE); mutationBuffer.putFloat(position + maxOffset, Float.NEGATIVE_INFINITY);
} }
@Override @Override

View File

@ -69,13 +69,13 @@ public class FloatDimensionIndexer implements DimensionIndexer<Float, Float, Flo
@Override @Override
public Float getMinValue() public Float getMinValue()
{ {
return Float.MIN_VALUE; return Float.NEGATIVE_INFINITY;
} }
@Override @Override
public Float getMaxValue() public Float getMaxValue()
{ {
return Float.MAX_VALUE; return Float.POSITIVE_INFINITY;
} }
@Override @Override

View File

@ -45,6 +45,25 @@ public class HistogramTest
Assert.assertEquals("histogram matches expected histogram", hExpected, h); Assert.assertEquals("histogram matches expected histogram", hExpected, h);
} }
/**
* This test differs from {@link #testOffer()} only in that it offers only negative values into Histogram. It's to
* expose the issue of using Float's MIN_VALUE that is actually positive as initial value for {@link Histogram#max}.
*/
@Test
public void testOfferOnlyNegative() {
final float[] values = {-0.3f, -.1f, -0.8f, -.7f, -.5f, -3f};
final float[] breaks = {-1f, -0.5f, 0.0f, 0.5f, 1f};
Histogram hExpected = new Histogram(breaks, new long[]{1,3,2,0,0,0}, -3f, -0.1f);
Histogram h = new Histogram(breaks);
for(float v : values) {
h.offer(v);
}
Assert.assertEquals("histogram matches expected histogram", hExpected, h);
}
@Test @Test
public void testToFromBytes() { public void testToFromBytes() {
float[] breaks = {-1f, -0.5f, 0.0f, 0.5f, 1f}; float[] breaks = {-1f, -0.5f, 0.0f, 0.5f, 1f};

View File

@ -70,8 +70,8 @@ public class CompressedFloatsSerdeTest
private final float values5[] = {123.16f, 1.12f, 62.00f, 462.12f, 517.71f, 56.54f, 971.32f, 824.22f, 472.12f, 625.26f}; private final float values5[] = {123.16f, 1.12f, 62.00f, 462.12f, 517.71f, 56.54f, 971.32f, 824.22f, 472.12f, 625.26f};
private final float values6[] = {1000000f, 1000001f, 1000002f, 1000003f, 1000004f, 1000005f, 1000006f, 1000007f, 1000008f}; private final float values6[] = {1000000f, 1000001f, 1000002f, 1000003f, 1000004f, 1000005f, 1000006f, 1000007f, 1000008f};
private final float values7[] = { private final float values7[] = {
Float.MAX_VALUE, Float.MIN_VALUE, 12378.5734f, -12718243.7496f, -93653653.1f, 12743153.385534f, 21431.414538f, Float.POSITIVE_INFINITY, Float.NEGATIVE_INFINITY, 12378.5734f, -12718243.7496f, -93653653.1f, 12743153.385534f,
65487435436632.123f, -43734526234564.65f 21431.414538f, 65487435436632.123f, -43734526234564.65f
}; };
public CompressedFloatsSerdeTest( public CompressedFloatsSerdeTest(

View File

@ -214,7 +214,7 @@ public class IncrementalIndexTest
Lists.newArrayList("string", "float", "long"), Lists.newArrayList("string", "float", "long"),
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"string", Arrays.asList("A", null, ""), "string", Arrays.asList("A", null, ""),
"float", Arrays.asList(Float.MAX_VALUE, null, ""), "float", Arrays.asList(Float.POSITIVE_INFINITY, null, ""),
"long", Arrays.asList(Long.MIN_VALUE, null, "") "long", Arrays.asList(Long.MIN_VALUE, null, "")
) )
) )
@ -223,7 +223,7 @@ public class IncrementalIndexTest
Row row = index.iterator().next(); Row row = index.iterator().next();
Assert.assertEquals(Arrays.asList(new String[]{"", "", "A"}), row.getRaw("string")); Assert.assertEquals(Arrays.asList(new String[]{"", "", "A"}), row.getRaw("string"));
Assert.assertEquals(Arrays.asList(new String[]{"", "", String.valueOf(Float.MAX_VALUE)}), row.getRaw("float")); Assert.assertEquals(Arrays.asList(new String[]{"", "", String.valueOf(Float.POSITIVE_INFINITY)}), row.getRaw("float"));
Assert.assertEquals(Arrays.asList(new String[]{"", "", String.valueOf(Long.MIN_VALUE)}), row.getRaw("long")); Assert.assertEquals(Arrays.asList(new String[]{"", "", String.valueOf(Long.MIN_VALUE)}), row.getRaw("long"));
} }