Skip to content

Commit

Permalink
Add double operator support for new NaN definition
Browse files Browse the repository at this point in the history
Add support for new nan defintion for =, <>, >, <, >=, <=, between, in,
not in.
  • Loading branch information
rschlussel committed Jun 6, 2024
1 parent 0b48e00 commit aa5de0c
Show file tree
Hide file tree
Showing 9 changed files with 582 additions and 127 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import static com.facebook.presto.common.type.RealType.REAL;
import static com.facebook.presto.common.type.SmallintType.SMALLINT;
import static com.facebook.presto.common.type.TinyintType.TINYINT;
import static java.lang.Double.doubleToLongBits;
import static java.lang.Float.intBitsToFloat;
import static java.lang.Math.toIntExact;
import static java.util.Locale.ENGLISH;
Expand Down Expand Up @@ -212,4 +213,49 @@ static <V> Map<String, V> normalizeEnumMap(Map<String, V> entries)
return entries.entrySet().stream()
.collect(toMap(e -> e.getKey().toUpperCase(ENGLISH), Map.Entry::getValue));
}

/**
* For our definitions of double and real, Nan=NaN is true
* NaN is greater than all other values, and +0=-0 is true.
* the below functions enforce that definition
*/
public static boolean doubleEquals(double a, double b)
{
// the first check ensures +0 == -0 is true. the second ensures that NaN == NaN is true
// for all other cases a == b and doubleToLongBits(a) == doubleToLongBits(b) will return
// the same result
// doubleToLongBits converts all NaNs to the same representation
return a == b || doubleToLongBits(a) == doubleToLongBits(b);
}

public static long doubleHashCode(double value)
{
// canonicalize +0 and -0 to a single value
value = value == -0 ? 0 : value;
// doubleToLongBits converts all NaNs to the same representation
return AbstractLongType.hash(doubleToLongBits(value));
}

public static int doubleCompare(double a, double b)
{
// these three ifs can only be true if neither value is NaN
if (a < b) {
return -1;
}
if (a > b) {
return 1;
}
// this check ensure doubleCompare(+0, -0) will return 0
// if we just did doubleToLongBits comparison, then they
// would not compare as equal
if (a == b) {
return 0;
}

// this ensures that doubleCompare(NaN, NaN) will return 0
// doubleToLongBits converts all NaNs to the same representation
long aBits = doubleToLongBits(a);
long bBits = doubleToLongBits(b);
return Long.compare(aBits, bBits);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,12 @@

import static com.facebook.presto.common.type.IntegerType.INTEGER;
import static com.facebook.presto.common.type.TypeUtils.containsDistinctType;
import static com.facebook.presto.common.type.TypeUtils.doubleCompare;
import static com.facebook.presto.common.type.TypeUtils.doubleEquals;
import static com.facebook.presto.common.type.TypeUtils.doubleHashCode;
import static com.facebook.presto.common.type.VarcharType.VARCHAR;
import static java.lang.Double.longBitsToDouble;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;

Expand Down Expand Up @@ -52,4 +57,29 @@ public void testContainsDistinctType()
assertTrue(containsDistinctType(ImmutableList.of(new ArrayType(new ArrayType(distinctType)))));
assertTrue(containsDistinctType(ImmutableList.of(new ArrayType(RowType.anonymous(ImmutableList.of(INTEGER, distinctType))))));
}

@Test
public void testDoubleHashCode()
{
assertEquals(doubleHashCode(0), doubleHashCode(Double.parseDouble("-0")));
//0x7ff8123412341234L is a different representation of NaN
assertEquals(doubleHashCode(Double.NaN), doubleHashCode(longBitsToDouble(0x7ff8123412341234L)));
}

@Test
public void testDoubleEquals()
{
assertTrue(doubleEquals(0, Double.parseDouble("-0")));
//0x7ff8123412341234L is a different representation of NaN
assertTrue(doubleEquals(Double.NaN, longBitsToDouble(0x7ff8123412341234L)));
}

@Test
public void testDoubleCompare()
{
assertEquals(doubleCompare(0, Double.parseDouble("-0")), 0);
assertEquals(doubleCompare(Double.NaN, Double.NaN), 0);
//0x7ff8123412341234L is a different representation of NaN
assertEquals(doubleCompare(Double.NaN, longBitsToDouble(0x7ff8123412341234L)), 0);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* Licensed 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 com.facebook.presto.hive;

import com.facebook.presto.testing.QueryRunner;
import com.facebook.presto.tests.AbstractTestNanQueries;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;

import java.util.Optional;

public class TestHiveDistributedNanQueries
extends AbstractTestNanQueries
{
@Override
protected QueryRunner createQueryRunner()
throws Exception
{
return HiveQueryRunner.createQueryRunner(ImmutableList.of(), ImmutableMap.of("use-new-nan-definition", "true"), ImmutableMap.of(), Optional.empty());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,7 @@
import com.facebook.presto.type.DateTimeOperators;
import com.facebook.presto.type.DecimalOperators;
import com.facebook.presto.type.DecimalParametricType;
import com.facebook.presto.type.DoubleComparisonOperators;
import com.facebook.presto.type.DoubleOperators;
import com.facebook.presto.type.EnumCasts;
import com.facebook.presto.type.HyperLogLogOperators;
Expand All @@ -255,6 +256,7 @@
import com.facebook.presto.type.IpAddressOperators;
import com.facebook.presto.type.IpPrefixOperators;
import com.facebook.presto.type.KllSketchOperators;
import com.facebook.presto.type.LegacyDoubleComparisonOperators;
import com.facebook.presto.type.LikeFunctions;
import com.facebook.presto.type.LongEnumOperators;
import com.facebook.presto.type.MapParametricType;
Expand Down Expand Up @@ -623,11 +625,11 @@ private void registerBuiltInTypes(FeaturesConfig featuresConfig)
addType(INTEGER);
addType(SMALLINT);
addType(TINYINT);
if(!featuresConfig.getUseNewNanDefinition()) {
if (!featuresConfig.getUseNewNanDefinition()) {
addType(OLD_NAN_DOUBLE);
addType(OLD_NAN_REAL);

} else {
}
else {
addType(DOUBLE);
addType(REAL);
}
Expand Down Expand Up @@ -798,9 +800,18 @@ private List<? extends SqlFunction> getBuiltInFunctions(FeaturesConfig featuresC
.scalar(SmallintOperators.SmallintDistinctFromOperator.class)
.scalars(TinyintOperators.class)
.scalar(TinyintOperators.TinyintDistinctFromOperator.class)
.scalars(DoubleOperators.class)
.scalar(DoubleOperators.DoubleDistinctFromOperator.class)
.scalars(RealOperators.class)
.scalars(DoubleOperators.class);

if (featuresConfig.getUseNewNanDefinition()) {
builder.scalars(DoubleComparisonOperators.class)
.scalar(DoubleComparisonOperators.DoubleDistinctFromOperator.class);
}
else {

builder.scalars(LegacyDoubleComparisonOperators.class)
.scalar(LegacyDoubleComparisonOperators.DoubleDistinctFromOperator.class);
}
builder.scalars(RealOperators.class)
.scalar(RealOperators.RealDistinctFromOperator.class)
.scalars(VarcharOperators.class)
.scalar(VarcharOperators.VarcharDistinctFromOperator.class)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,141 @@
/*
* Licensed 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 com.facebook.presto.type;

import com.facebook.presto.common.block.Block;
import com.facebook.presto.common.type.StandardTypes;
import com.facebook.presto.spi.function.BlockIndex;
import com.facebook.presto.spi.function.BlockPosition;
import com.facebook.presto.spi.function.IsNull;
import com.facebook.presto.spi.function.ScalarOperator;
import com.facebook.presto.spi.function.SqlNullable;
import com.facebook.presto.spi.function.SqlType;

import static com.facebook.presto.common.function.OperatorType.BETWEEN;
import static com.facebook.presto.common.function.OperatorType.EQUAL;
import static com.facebook.presto.common.function.OperatorType.GREATER_THAN;
import static com.facebook.presto.common.function.OperatorType.GREATER_THAN_OR_EQUAL;
import static com.facebook.presto.common.function.OperatorType.HASH_CODE;
import static com.facebook.presto.common.function.OperatorType.IS_DISTINCT_FROM;
import static com.facebook.presto.common.function.OperatorType.LESS_THAN;
import static com.facebook.presto.common.function.OperatorType.LESS_THAN_OR_EQUAL;
import static com.facebook.presto.common.function.OperatorType.NOT_EQUAL;
import static com.facebook.presto.common.type.DoubleType.DOUBLE;
import static com.facebook.presto.common.type.TypeUtils.doubleCompare;
import static com.facebook.presto.common.type.TypeUtils.doubleEquals;
import static com.facebook.presto.common.type.TypeUtils.doubleHashCode;

public class DoubleComparisonOperators
{
private DoubleComparisonOperators() {}

@ScalarOperator(EQUAL)
@SuppressWarnings("FloatingPointEquality")
@SqlType(StandardTypes.BOOLEAN)
@SqlNullable
public static Boolean equal(@SqlType(StandardTypes.DOUBLE) double left, @SqlType(StandardTypes.DOUBLE) double right)
{
return doubleEquals(left, right);
}

@ScalarOperator(NOT_EQUAL)
@SuppressWarnings("FloatingPointEquality")
@SqlType(StandardTypes.BOOLEAN)
@SqlNullable
public static Boolean notEqual(@SqlType(StandardTypes.DOUBLE) double left, @SqlType(StandardTypes.DOUBLE) double right)
{
return !equal(left, right);
}

@ScalarOperator(LESS_THAN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean lessThan(@SqlType(StandardTypes.DOUBLE) double left, @SqlType(StandardTypes.DOUBLE) double right)
{
return doubleCompare(left, right) < 0;
}

@ScalarOperator(LESS_THAN_OR_EQUAL)
@SqlType(StandardTypes.BOOLEAN)
public static boolean lessThanOrEqual(@SqlType(StandardTypes.DOUBLE) double left, @SqlType(StandardTypes.DOUBLE) double right)
{
return doubleCompare(left, right) <= 0;
}

@ScalarOperator(GREATER_THAN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean greaterThan(@SqlType(StandardTypes.DOUBLE) double left, @SqlType(StandardTypes.DOUBLE) double right)
{
return doubleCompare(left, right) == 1;
}

@ScalarOperator(GREATER_THAN_OR_EQUAL)
@SqlType(StandardTypes.BOOLEAN)
public static boolean greaterThanOrEqual(@SqlType(StandardTypes.DOUBLE) double left, @SqlType(StandardTypes.DOUBLE) double right)
{
return doubleCompare(left, right) >= 0;
}

@ScalarOperator(BETWEEN)
@SqlType(StandardTypes.BOOLEAN)
public static boolean between(@SqlType(StandardTypes.DOUBLE) double value, @SqlType(StandardTypes.DOUBLE) double min, @SqlType(StandardTypes.DOUBLE) double max)
{
return lessThanOrEqual(min, value) && lessThanOrEqual(value, max);
}

@ScalarOperator(HASH_CODE)
@SqlType(StandardTypes.BIGINT)
public static long hashCode(@SqlType(StandardTypes.DOUBLE) double value)
{
return doubleHashCode(value);
}

@ScalarOperator(IS_DISTINCT_FROM)
public static class DoubleDistinctFromOperator
{
@SqlType(StandardTypes.BOOLEAN)
public static boolean isDistinctFrom(
@SqlType(StandardTypes.DOUBLE) double left,
@IsNull boolean leftNull,
@SqlType(StandardTypes.DOUBLE) double right,
@IsNull boolean rightNull)
{
if (leftNull != rightNull) {
return true;
}
if (leftNull) {
return false;
}
return notEqual(left, right);
}

@SqlType(StandardTypes.BOOLEAN)
public static boolean isDistinctFrom(
@BlockPosition @SqlType(value = StandardTypes.DOUBLE, nativeContainerType = double.class) Block leftBlock,
@BlockIndex int leftPosition,
@BlockPosition @SqlType(value = StandardTypes.DOUBLE, nativeContainerType = double.class) Block rightBlock,
@BlockIndex int rightPosition)
{
if (leftBlock.isNull(leftPosition) != rightBlock.isNull(rightPosition)) {
return true;
}
if (leftBlock.isNull(leftPosition)) {
return false;
}
double left = DOUBLE.getDouble(leftBlock, leftPosition);
double right = DOUBLE.getDouble(rightBlock, rightPosition);
return notEqual(left, right);
}
}
}
Loading

0 comments on commit aa5de0c

Please sign in to comment.