[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2019-01-08 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r245973381
 
 

 ##
 File path: 
sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
 ##
 @@ -724,17 +724,29 @@ class DataFrameAggregateSuite extends QueryTest with 
SharedSQLContext {
 "type: GroupBy]"))
   }
 
-  test("SPARK-26021: Double and Float 0.0/-0.0 should be equal when grouping") 
{
-val colName = "i"
-val doubles = Seq(0.0d, -0.0d, 
0.0d).toDF(colName).groupBy(colName).count().collect()
-val floats = Seq(0.0f, -0.0f, 
0.0f).toDF(colName).groupBy(colName).count().collect()
-
-assert(doubles.length == 1)
-assert(floats.length == 1)
-// using compare since 0.0 == -0.0 is true
-assert(java.lang.Double.compare(doubles(0).getDouble(0), 0.0d) == 0)
-assert(java.lang.Float.compare(floats(0).getFloat(0), 0.0f) == 0)
-assert(doubles(0).getLong(1) == 3)
-assert(floats(0).getLong(1) == 3)
+  test("SPARK-26021: NaN and -0.0 in grouping expressions") {
+checkAnswer(
+  Seq(0.0f, -0.0f, 0.0f/0.0f, Float.NaN).toDF("f").groupBy("f").count(),
 
 Review comment:
   fixed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2019-01-08 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r245946612
 
 

 ##
 File path: docs/sql-migration-guide-upgrade.md
 ##
 @@ -25,7 +25,7 @@ displayTitle: Spark SQL Upgrading Guide
 
   - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a 
grouped dataset with key attribute wrongly named as "value", if the key is 
non-struct type, e.g. int, string, array, etc. This is counterintuitive and 
makes the schema of aggregation queries weird. For example, the schema of 
`ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the 
grouping attribute to "key". The old behaviour is preserved under a newly added 
configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a 
default value of `false`.
 
-  - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal 
to 0.0, but users can still distinguish them via `Dataset.show`, 
`Dataset.collect` etc. Since Spark 3.0, float/double -0.0 is replaced by 0.0 
internally, and users can't distinguish them any more.
+  - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal 
to 0.0, but -0.0 and 0.0 are considered as different values when used in 
aggregate grouping keys, window partition keys and join keys. Since Spark 3.0, 
this bug is fixed. For example, `Seq(-0.0, 0.0).toDF("d").groupBy("d").count()` 
returns `[(0.0, 2)]` in Spark 3.0, and `[(0.0, 1), (-0.0, 1)]` in Spark 2.4 and 
earlier.
 
 Review comment:
   I think we only need to mention the difference between new and old versions.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2019-01-07 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r245866133
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala
 ##
 @@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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 org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, 
CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, 
LambdaFunction, NamedLambdaVariable, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, 
Window}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types._
+
+/**
+ * We need to take care of special floating numbers (NaN and -0.0) in several 
places:
+ *   1. When compare values, different NaNs should be treated as same, `-0.0` 
and `0.0` should be
+ *  treated as same.
+ *   2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 
0.0 should belong
+ *  to the same group.
+ *   3. In join keys, different NaNs should be treated as same, `-0.0` and 
`0.0` should be
+ *  treated as same.
+ *   4. In window partition keys, different NaNs should be treated as same, 
`-0.0` and `0.0`
+ *  should be treated as same.
+ *
+ * Case 1 is fine, as we handle NaN and -0.0 well during comparison. For 
complex types, we
+ * recursively compare the fields/elements, so it's also fine.
+ *
+ * Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary 
directly, and different
+ * NaNs have different binary representation, and the same thing happens for 
-0.0 and 0.0.
+ *
+ * This rule normalizes NaN and -0.0 in Window partition keys, Join keys and 
Aggregate grouping
+ * expressions.
+ *
+ * Note that, this rule should be an analyzer rule, as it must be applied to 
make the query result
+ * corrected. Currently it's executed as an optimizer rule, because the 
optimizer may create new
+ * joins(for subquery) and reorder joins(may change the join condition), and 
this rule needs to be
+ * executed at the end.
+ */
+object NormalizeFloatingNumbers extends Rule[LogicalPlan] {
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan match {
+// A subquery will be rewritten into join later, and will go through this 
rule
+// eventually. Here we skip subquery, as we only need to run this rule 
once.
+case _: Subquery => plan
+
+case _ => plan transform {
+  case w: Window if w.partitionSpec.exists(p => needNormalize(p.dataType)) 
=>
+w.copy(partitionSpec = w.partitionSpec.map(normalize))
+
+  case j @ ExtractEquiJoinKeys(_, leftKeys, rightKeys, condition, _, _)
+if leftKeys.exists(k => needNormalize(k.dataType)) =>
 
 Review comment:
   the analyzer will make sure the left and right join keys are of the same 
data type. I'll add a comment to explain it, thanks!


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2019-01-07 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r245866178
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala
 ##
 @@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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 org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, 
CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, 
LambdaFunction, NamedLambdaVariable, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, 
Window}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types._
+
+/**
+ * We need to take care of special floating numbers (NaN and -0.0) in several 
places:
+ *   1. When compare values, different NaNs should be treated as same, `-0.0` 
and `0.0` should be
+ *  treated as same.
+ *   2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 
0.0 should belong
+ *  to the same group.
+ *   3. In join keys, different NaNs should be treated as same, `-0.0` and 
`0.0` should be
+ *  treated as same.
+ *   4. In window partition keys, different NaNs should be treated as same, 
`-0.0` and `0.0`
+ *  should be treated as same.
+ *
+ * Case 1 is fine, as we handle NaN and -0.0 well during comparison. For 
complex types, we
+ * recursively compare the fields/elements, so it's also fine.
+ *
+ * Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary 
directly, and different
+ * NaNs have different binary representation, and the same thing happens for 
-0.0 and 0.0.
+ *
+ * This rule normalizes NaN and -0.0 in Window partition keys, Join keys and 
Aggregate grouping
+ * expressions.
+ *
+ * Note that, this rule should be an analyzer rule, as it must be applied to 
make the query result
+ * corrected. Currently it's executed as an optimizer rule, because the 
optimizer may create new
+ * joins(for subquery) and reorder joins(may change the join condition), and 
this rule needs to be
+ * executed at the end.
+ */
+object NormalizeFloatingNumbers extends Rule[LogicalPlan] {
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan match {
+// A subquery will be rewritten into join later, and will go through this 
rule
+// eventually. Here we skip subquery, as we only need to run this rule 
once.
+case _: Subquery => plan
+
+case _ => plan transform {
+  case w: Window if w.partitionSpec.exists(p => needNormalize(p.dataType)) 
=>
+w.copy(partitionSpec = w.partitionSpec.map(normalize))
+
+  case j @ ExtractEquiJoinKeys(_, leftKeys, rightKeys, condition, _, _)
+if leftKeys.exists(k => needNormalize(k.dataType)) =>
+val newLeftJoinKeys = leftKeys.map(normalize)
+val newRightJoinKeys = rightKeys.map(normalize)
+val newConditions = newLeftJoinKeys.zip(newRightJoinKeys).map {
+  case (l, r) => EqualTo(l, r)
+} ++ condition
+j.copy(condition = Some(newConditions.reduce(And)))
+
+  // TODO: ideally Aggregate should also be handled here, but its grouping 
expressions are
+  // mixed in its aggregate expressions. It's unreliable to change the 
grouping expressions
+  // here. For now we normalize grouping expressions in `AggUtils` during 
planning.
+}
+  }
+
+  private def needNormalize(dt: DataType): Boolean = dt match {
+case FloatType | DoubleType => true
+case StructType(fields) => fields.exists(f => needNormalize(f.dataType))
+case ArrayType(et, _) => needNormalize(et)
+// We don't need to handle MapType here, as it's not comparable.
 
 Review comment:
   good point! I'll update soon.


This is an automated message from the Apache Git Service.
To respond to the message, please log on 

[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2019-01-07 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r245865987
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala
 ##
 @@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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 org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, 
CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, 
LambdaFunction, NamedLambdaVariable, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, 
Window}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types._
+
+/**
+ * We need to take care of special floating numbers (NaN and -0.0) in several 
places:
+ *   1. When compare values, different NaNs should be treated as same, `-0.0` 
and `0.0` should be
+ *  treated as same.
+ *   2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 
0.0 should belong
+ *  to the same group.
+ *   3. In join keys, different NaNs should be treated as same, `-0.0` and 
`0.0` should be
 
 Review comment:
   That a good point. In Spark SQL, the EQUAL operator thinks 0.0 and -0.0 are 
same, so we have to follow it in join keys. I'm not sure how the SQL standard 
defines it, but it's another topic if we want to change the equal semantic of 
Spark SQL.
   
   But you are right that we don't have to do it for join, we only need to do 
normalization for certain types of join that do binary comparison.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2019-01-01 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244656211
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala
 ##
 @@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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 org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, 
CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, 
LambdaFunction, NamedLambdaVariable, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, 
Window}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types._
+
+/**
+ * We need to take care of special floating numbers (NaN and -0.0) in several 
places:
+ *   1. When compare values, different NaNs should be treated as same, `-0.0` 
and `0.0` should be
+ *  treated as same.
+ *   2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 
0.0 should belong
+ *  to the same group.
+ *   3. In join keys, different NaNs should be treated as same, `-0.0` and 
`0.0` should be
+ *  treated as same.
+ *   4. In window partition keys, different NaNs should be treated as same, 
`-0.0` and `0.0`
+ *  should be treated as same.
+ *
+ * Case 1 is fine, as we handle NaN and -0.0 well during comparison. For 
complex types, we
+ * recursively compare the fields/elements, so it's also fine.
+ *
+ * Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary 
directly, and different
+ * NaNs have different binary representation, and the same thing happens for 
-0.0 and 0.0.
+ *
+ * This rule normalizes NaN and -0.0 in Window partition keys, Join keys and 
Aggregate grouping
+ * expressions.
+ *
+ * Note that, this rule should be an analyzer rule, as it must be applied to 
make the query result
+ * corrected. Currently it's executed as an optimizer rule, because the 
optimizer may create new
+ * joins(for subquery) and reorder joins(may change the join condition), and 
this rule needs to be
+ * executed at the end.
+ */
+object NormalizeFloatingNumbers extends Rule[LogicalPlan] {
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan match {
+// A subquery will be rewritten into join later, and will go through this 
rule
 
 Review comment:
   This is same as `ExtractPythonUDFs`


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2019-01-01 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244655875
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
 ##
 @@ -102,8 +102,8 @@ object ExtractEquiJoinKeys extends Logging with 
PredicateHelper {
   type ReturnType =
 (JoinType, Seq[Expression], Seq[Expression], Option[Expression], 
LogicalPlan, LogicalPlan)
 
-  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
-case join @ Join(left, right, joinType, condition) =>
+  def unapply(join: Join): Option[ReturnType] = join match {
+case Join(left, right, joinType, condition) =>
 
 Review comment:
   we can, but that will introduce a lot of code diff, because of the 
indentation...


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2019-01-01 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244655849
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala
 ##
 @@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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 org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, 
CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, 
LambdaFunction, NamedLambdaVariable, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, 
Window}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types._
+
+/**
+ * We need to take care of special floating numbers (NaN and -0.0) in several 
places:
+ *   1. When compare values, different NaNs should be treated as same, `-0.0` 
and `0.0` should be
+ *  treated as same.
+ *   2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 
0.0 should belong
+ *  to the same group.
+ *   3. In join keys, different NaNs should be treated as same, `-0.0` and 
`0.0` should be
+ *  treated as same.
+ *   4. In window partition keys, different NaNs should be treated as same, 
`-0.0` and `0.0`
+ *  should be treated as same.
+ *
+ * Case 1 is fine, as we handle NaN and -0.0 well during comparison. For 
complex types, we
+ * recursively compare the fields/elements, so it's also fine.
+ *
+ * Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary 
directly, and different
+ * NaNs have different binary representation, and the same thing happens for 
-0.0 and 0.0.
+ *
+ * This rule normalizes NaN and -0.0 in Window partition keys, Join keys and 
Aggregate grouping
+ * expressions.
+ *
+ * Note that, this rule should be an analyzer rule, as it must be applied to 
make the query result
+ * corrected. Currently it's executed as an optimizer rule, because the 
optimizer may create new
+ * joins(for subquery) and reorder joins(may change the join condition), and 
this rule needs to be
+ * executed at the end.
+ */
+object NormalizeFloatingNumbers extends Rule[LogicalPlan] {
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan match {
+// A subquery will be rewritten into join later, and will go through this 
rule
 
 Review comment:
   `OptimizeSubqueries` will apply the entire optimizer and triggers this rule.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2019-01-01 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244655827
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala
 ##
 @@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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 org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, 
CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, 
LambdaFunction, NamedLambdaVariable, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, 
Window}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types._
+
+/**
+ * We need to take care of special floating numbers (NaN and -0.0) in several 
places:
+ *   1. When compare values, different NaNs should be treated as same, `-0.0` 
and `0.0` should be
+ *  treated as same.
+ *   2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 
0.0 should belong
+ *  to the same group.
+ *   3. In join keys, different NaNs should be treated as same, `-0.0` and 
`0.0` should be
+ *  treated as same.
+ *   4. In window partition keys, different NaNs should be treated as same, 
`-0.0` and `0.0`
+ *  should be treated as same.
+ *
+ * Case 1 is fine, as we handle NaN and -0.0 well during comparison. For 
complex types, we
+ * recursively compare the fields/elements, so it's also fine.
+ *
+ * Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary 
directly, and different
+ * NaNs have different binary representation, and the same thing happens for 
-0.0 and 0.0.
+ *
+ * This rule normalizes NaN and -0.0 in Window partition keys, Join keys and 
Aggregate grouping
+ * expressions.
+ */
+object NormalizeFloatingNumbers extends Rule[LogicalPlan] {
 
 Review comment:
   ah good catch!


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-29 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244524354
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala
 ##
 @@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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 org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, 
CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, 
LambdaFunction, NamedLambdaVariable, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, 
Window}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types._
+
+/**
+ * We need to take care of special floating numbers (NaN and -0.0) in several 
places:
+ *   1. When compare values, different NaNs should be treated as same, `-0.0` 
and `0.0` should be
+ *  treated as same.
+ *   2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 
0.0 should belong
+ *  to the same group.
+ *   3. In join keys, different NaNs should be treated as same, `-0.0` and 
`0.0` should be
+ *  treated as same.
+ *   4. In window partition keys, different NaNs should be treated as same, 
`-0.0` and `0.0`
+ *  should be treated as same.
+ *
+ * Case 1 is fine, as we handle NaN and -0.0 well during comparison. For 
complex types, we
+ * recursively compare the fields/elements, so it's also fine.
+ *
+ * Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary 
directly, and different
+ * NaNs have different binary representation, and the same thing happens for 
-0.0 and 0.0.
+ *
+ * This rule normalizes NaN and -0.0 in Window partition keys, Join keys and 
Aggregate grouping
+ * expressions.
+ */
+object NormalizeFloatingNumbers extends Rule[LogicalPlan] {
 
 Review comment:
   The major reason is we create `Join`s during optimizaiton (for subquery), 
and I'm also worried about join reorder may break it. I'll add comment for it. 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-29 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244524335
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala
 ##
 @@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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 org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, 
CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, 
LambdaFunction, NamedLambdaVariable, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, 
Window}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types._
+
+/**
+ * We need to take care of special floating numbers (NaN and -0.0) in several 
places:
+ *   1. When compare values, different NaNs should be treated as same, `-0.0` 
and `0.0` should be
+ *  treated as same.
+ *   2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 
0.0 should belong
+ *  to the same group.
+ *   3. In join keys, different NaNs should be treated as same, `-0.0` and 
`0.0` should be
+ *  treated as same.
+ *   4. In window partition keys, different NaNs should be treated as same, 
`-0.0` and `0.0`
+ *  should be treated as same.
+ *
+ * Case 1 is fine, as we handle NaN and -0.0 well during comparison. For 
complex types, we
+ * recursively compare the fields/elements, so it's also fine.
+ *
+ * Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary 
directly, and different
+ * NaNs have different binary representation, and the same thing happens for 
-0.0 and 0.0.
+ *
+ * This rule normalizes NaN and -0.0 in Window partition keys, Join keys and 
Aggregate grouping
+ * expressions.
+ */
+object NormalizeFloatingNumbers extends Rule[LogicalPlan] {
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan match {
+// A subquery will be rewritten into join later, and will go through this 
rule
+// eventually. Here we skip subquery, as we only need to run this rule 
once.
+case _: Subquery => plan
+
+case _ => plan transform {
+  case w: Window if w.partitionSpec.exists(p => needNormalize(p.dataType)) 
=>
+w.copy(partitionSpec = w.partitionSpec.map(normalize))
 
 Review comment:
   assume the query is `select a, a + sum(a) over (partition by a) ...`.
   
   Since the project list is evaluated for each input row, I think the `a` in 
the project list should retain the different of -0.0. Thus I think only 
`partitionSpec` needs to be normalized.
   
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-29 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244522622
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
 ##
 @@ -102,8 +102,8 @@ object ExtractEquiJoinKeys extends Logging with 
PredicateHelper {
   type ReturnType =
 (JoinType, Seq[Expression], Seq[Expression], Option[Expression], 
LogicalPlan, LogicalPlan)
 
-  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
-case join @ Join(left, right, joinType, condition) =>
+  def unapply(join: Join): Option[ReturnType] = join match {
+case Join(left, right, joinType, condition) =>
 
 Review comment:
   It's same as before, except that, `j @ ExtractEquiJoinKeys...` the `j` 
becomes `Join` type.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-28 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244327980
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala
 ##
 @@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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 org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, 
CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, 
LambdaFunction, NamedLambdaVariable, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, 
Window}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types._
+
+/**
+ * We need to take care of special floating numbers (NaN and -0.0) in several 
places:
+ *   1. When compare values, different NaNs should be treated as same, `-0.0` 
and `0.0` should be
+ *  treated as same.
+ *   2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 
0.0 should belong
+ *  to the same group.
+ *   3. In join keys, different NaNs should be treated as same, `-0.0` and 
`0.0` should be
+ *  treated as same.
+ *   4. In window partition keys, different NaNs should be treated as same, 
`-0.0` and `0.0`
+ *  should be treated as same.
+ *
+ * Case 1 is fine, as we handle NaN and -0.0 well during comparison. For 
complex types, we
+ * recursively compare the fields/elements, so it's also fine.
+ *
+ * Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary 
directly, and different
+ * NaNs have different binary representation, and the same thing happens for 
-0.0 and 0.0.
+ *
+ * This rule normalizes NaN and -0.0 in Window partition keys, Join keys and 
Aggregate grouping
+ * expressions.
+ */
+object NormalizeFloatingNumbers extends Rule[LogicalPlan] {
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan match {
+// A subquery will be rewritten into join later, and will go through this 
rule
+// eventually. Here we skip subquery, as we only need to run this rule 
once.
+case _: Subquery => plan
+
+case _ => plan transform {
+  case w: Window if w.partitionSpec.exists(p => needNormalize(p.dataType)) 
=>
+w.copy(partitionSpec = w.partitionSpec.map(normalize))
+
+  case j @ ExtractEquiJoinKeys(_, leftKeys, rightKeys, condition, _, _)
+if leftKeys.exists(k => needNormalize(k.dataType)) =>
+val newLeftJoinKeys = leftKeys.map(normalize)
+val newRightJoinKeys = rightKeys.map(normalize)
+val newConditions = newLeftJoinKeys.zip(newRightJoinKeys).map {
+  case (l, r) => EqualTo(l, r)
+} ++ condition
+j.copy(condition = Some(newConditions.reduce(And)))
+
+  // TODO: ideally Aggregate should also be handled here, but its grouping 
expressions are
+  // mixed in its aggregate expressions. It's unreliable to change the 
grouping expressions
+  // here. For now we normalize grouping expressions in `AggUtils` during 
planning.
+}
+  }
+
+  private def needNormalize(dt: DataType): Boolean = dt match {
+case FloatType | DoubleType => true
+case StructType(fields) => fields.exists(f => needNormalize(f.dataType))
+case ArrayType(et, _) => needNormalize(et)
+// We don't need to handle MapType here, as it's not comparable.
+case _ => false
+  }
+
+  private[sql] def normalize(expr: Expression): Expression = expr match {
+case _ if expr.dataType == FloatType || expr.dataType == DoubleType =>
+  NormalizeNaNAndZero(expr)
+
+case CreateNamedStruct(children) =>
+  CreateNamedStruct(children.map(normalize))
+
+case CreateNamedStructUnsafe(children) =>
+  CreateNamedStructUnsafe(children.map(normalize))
+
+case CreateArray(children) =>
+  CreateArray(children.map(normalize))
+
+case CreateMap(children) =>
+  

[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-27 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244163336
 
 

 ##
 File path: docs/sql-migration-guide-upgrade.md
 ##
 @@ -25,7 +25,7 @@ displayTitle: Spark SQL Upgrading Guide
 
   - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a 
grouped dataset with key attribute wrongly named as "value", if the key is 
non-struct type, e.g. int, string, array, etc. This is counterintuitive and 
makes the schema of aggregation queries weird. For example, the schema of 
`ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the 
grouping attribute to "key". The old behaviour is preserved under a newly added 
configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a 
default value of `false`.
 
-  - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal 
to 0.0, but users can still distinguish them via `Dataset.show`, 
`Dataset.collect` etc. Since Spark 3.0, float/double -0.0 is replaced by 0.0 
internally, and users can't distinguish them any more.
+  - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal 
to 0.0, but -0.0 and 0.0 are considered as different values when used in 
aggregate grouping keys, window partition keys and join keys. Since Spark 3.0, 
this bug is fixed. For example, `Seq(-0.0, 0.0).toDF("d").groupBy("d").count()` 
returns `[(0.0, 2)]` in Spark 3.0, and `[(0.0, 1), (-0.0, 1)]` in Spark 2.4 and 
earlier.
 
 Review comment:
   I keep this migration guide because this bug is not very intuitive: 
literally -0.0 is not 0.0.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-27 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244135560
 
 

 ##
 File path: docs/sql-migration-guide-upgrade.md
 ##
 @@ -25,8 +25,6 @@ displayTitle: Spark SQL Upgrading Guide
 
   - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a 
grouped dataset with key attribute wrongly named as "value", if the key is 
non-struct type, e.g. int, string, array, etc. This is counterintuitive and 
makes the schema of aggregation queries weird. For example, the schema of 
`ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the 
grouping attribute to "key". The old behaviour is preserved under a newly added 
configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a 
default value of `false`.
 
-  - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal 
to 0.0, but users can still distinguish them via `Dataset.show`, 
`Dataset.collect` etc. Since Spark 3.0, float/double -0.0 is replaced by 0.0 
internally, and users can't distinguish them any more.
 
 Review comment:
   ah i see what you mean. Are you saying we should add migration guide for the 
behavior changes of grouping key/window partition key?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-27 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244134175
 
 

 ##
 File path: docs/sql-migration-guide-upgrade.md
 ##
 @@ -25,8 +25,6 @@ displayTitle: Spark SQL Upgrading Guide
 
   - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a 
grouped dataset with key attribute wrongly named as "value", if the key is 
non-struct type, e.g. int, string, array, etc. This is counterintuitive and 
makes the schema of aggregation queries weird. For example, the schema of 
`ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the 
grouping attribute to "key". The old behaviour is preserved under a newly added 
configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a 
default value of `false`.
 
-  - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal 
to 0.0, but users can still distinguish them via `Dataset.show`, 
`Dataset.collect` etc. Since Spark 3.0, float/double -0.0 is replaced by 0.0 
internally, and users can't distinguish them any more.
 
 Review comment:
   yes, and it's a bug. But if -0.0 is not used in grouping keys(and other 
similar places), users should still be able to distinguish it. 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-26 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244092877
 
 

 ##
 File path: docs/sql-migration-guide-upgrade.md
 ##
 @@ -25,8 +25,6 @@ displayTitle: Spark SQL Upgrading Guide
 
   - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a 
grouped dataset with key attribute wrongly named as "value", if the key is 
non-struct type, e.g. int, string, array, etc. This is counterintuitive and 
makes the schema of aggregation queries weird. For example, the schema of 
`ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the 
grouping attribute to "key". The old behaviour is preserved under a newly added 
configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a 
default value of `false`.
 
-  - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal 
to 0.0, but users can still distinguish them via `Dataset.show`, 
`Dataset.collect` etc. Since Spark 3.0, float/double -0.0 is replaced by 0.0 
internally, and users can't distinguish them any more.
 
 Review comment:
   checkout the [test 
case](https://github.com/apache/spark/pull/23388/files#diff-4c0b1f729d651b04f14e72260555f623R397),
 "distinguish -0.0" is not about agg or join.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-26 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244084868
 
 

 ##
 File path: docs/sql-migration-guide-upgrade.md
 ##
 @@ -25,8 +25,6 @@ displayTitle: Spark SQL Upgrading Guide
 
   - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a 
grouped dataset with key attribute wrongly named as "value", if the key is 
non-struct type, e.g. int, string, array, etc. This is counterintuitive and 
makes the schema of aggregation queries weird. For example, the schema of 
`ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the 
grouping attribute to "key". The old behaviour is preserved under a newly added 
configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a 
default value of `false`.
 
-  - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal 
to 0.0, but users can still distinguish them via `Dataset.show`, 
`Dataset.collect` etc. Since Spark 3.0, float/double -0.0 is replaced by 0.0 
internally, and users can't distinguish them any more.
 
 Review comment:
   yea


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-26 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244084140
 
 

 ##
 File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
 ##
 @@ -1757,7 +1757,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
   sql("insert into tbl values ('3', '2.3')")
   checkAnswer(
 sql("select (cast (99 as decimal(19,6)) + cast('3' as decimal)) * 
cast('2.3' as decimal)"),
-Row(204.0)
+Row(BigDecimal(204.0))
 
 Review comment:
   It does not.
   
   This test case was wrong at the first place, my change to the `checkAnswer` 
expose it.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-26 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244080056
 
 

 ##
 File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
 ##
 @@ -1757,7 +1757,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
   sql("insert into tbl values ('3', '2.3')")
   checkAnswer(
 sql("select (cast (99 as decimal(19,6)) + cast('3' as decimal)) * 
cast('2.3' as decimal)"),
-Row(204.0)
+Row(BigDecimal(204.0))
 
 Review comment:
   the result is decimal type, not double.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-26 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244079988
 
 

 ##
 File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
 ##
 @@ -141,11 +141,13 @@ class HiveUDFSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils {
   }
 
   test("SPARK-2693 udaf aggregates test") {
-checkAnswer(sql("SELECT percentile(key, 1) FROM src LIMIT 1"),
-  sql("SELECT max(key) FROM src").collect().toSeq)
+checkAnswer(
+  sql("SELECT percentile(key, 1) FROM src LIMIT 1"),
+  sql("SELECT double(max(key)) FROM src"))
 
 Review comment:
   `percentile` always return  double, we need to cast `max` to double so that 
we can compare the results.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-26 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244079922
 
 

 ##
 File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
 ##
 @@ -594,7 +594,7 @@ abstract class AggregationQuerySuite extends QueryTest 
with SQLTestUtils with Te
   |  max(distinct value1)
   |FROM agg2
 """.stripMargin),
-  Row(-60, 70.0, 101.0/9.0, 5.6, 100))
+  Row(-60, 70, 101.0/9.0, 5.6, 100))
 
 Review comment:
   sum of int is long, we shouldn't use double here.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-26 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244079883
 
 

 ##
 File path: 
sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala
 ##
 @@ -397,27 +397,42 @@ class DatasetPrimitiveSuite extends QueryTest with 
SharedSQLContext {
   test("special floating point values") {
 import org.scalatest.exceptions.TestFailedException
 
-// Spark treats -0.0 as 0.0
+// Spark distinguishes -0.0 and 0.0
 intercept[TestFailedException] {
-  checkDataset(Seq(-0.0d).toDS(), -0.0d)
+  checkDataset(Seq(-0.0d).toDS(), 0.0d)
 
 Review comment:
   This is to prove that the test framework can distinguish -0.0 and 0.0.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-26 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244031793
 
 

 ##
 File path: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java
 ##
 @@ -198,46 +198,11 @@ protected final void writeLong(long offset, long value) {
 Platform.putLong(getBuffer(), offset, value);
   }
 
-  // We need to take care of NaN and -0.0 in several places:
 
 Review comment:
   The comments are moved to the new rule.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-26 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244031469
 
 

 ##
 File path: sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
 ##
 @@ -124,24 +124,6 @@ abstract class QueryTest extends PlanTest {
 }
   }
 
-  private def compare(obj1: Any, obj2: Any): Boolean = (obj1, obj2) match {
 
 Review comment:
   moved to the object, so that we can reuse it.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] retain the difference between 0.0 and -0.0

2018-12-26 Thread GitBox
cloud-fan commented on a change in pull request #23388: [SPARK-26448][SQL] 
retain the difference between 0.0 and -0.0
URL: https://github.com/apache/spark/pull/23388#discussion_r244031389
 
 

 ##
 File path: 
sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
 ##
 @@ -295,16 +295,4 @@ class DataFrameJoinSuite extends QueryTest with 
SharedSQLContext {
   df.join(df, df("id") <=> df("id")).queryExecution.optimizedPlan
 }
   }
-
-  test("NaN and -0.0 in join keys") {
 
 Review comment:
   moved to `JoinSuite`.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org