You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "aishikbh (via GitHub)" <gi...@apache.org> on 2023/11/20 14:12:59 UTC

[PR] [bugfix] Add new Transformer to transform negative zero and NaN behaviour [pinot]

aishikbh opened a new pull request, #12032:
URL: https://github.com/apache/pinot/pull/12032

   This PR transforms the values of `-0.0` to `0.0` and `NaN` to default null. Allowing `-0.0` and `NaN` messes up the binary search/sorting in certain cases. Further discussion [here](https://github.com/apache/pinot/issues/10697).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #12032:
URL: https://github.com/apache/pinot/pull/12032#issuecomment-1824419001

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/12032?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   Attention: `46 lines` in your changes are missing coverage. Please review.
   > Comparison is base [(`32db500`)](https://app.codecov.io/gh/apache/pinot/commit/32db50016011bc64b9548438e9d1df0e266e27a0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) 61.62% compared to head [(`3d34dfa`)](https://app.codecov.io/gh/apache/pinot/pull/12032?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) 34.94%.
   > Report is 10 commits behind head on master.
   
   | [Files](https://app.codecov.io/gh/apache/pinot/pull/12032?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Patch % | Lines |
   |---|---|---|
   | [...cal/recordtransformer/SpecialValueTransformer.java](https://app.codecov.io/gh/apache/pinot/pull/12032?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWNvcmR0cmFuc2Zvcm1lci9TcGVjaWFsVmFsdWVUcmFuc2Zvcm1lci5qYXZh) | 0.00% | [44 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12032?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   | [.../local/recordtransformer/CompositeTransformer.java](https://app.codecov.io/gh/apache/pinot/pull/12032?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWNvcmR0cmFuc2Zvcm1lci9Db21wb3NpdGVUcmFuc2Zvcm1lci5qYXZh) | 0.00% | [2 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12032?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   
   <details><summary>Additional details and impacted files</summary>
   
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #12032       +/-   ##
   =============================================
   - Coverage     61.62%   34.94%   -26.69%     
   + Complexity     1151        6     -1145     
   =============================================
     Files          2385     2310       -75     
     Lines        129423   125821     -3602     
     Branches      20036    19495      -541     
   =============================================
   - Hits          79751    43962    -35789     
   - Misses        43855    78745    +34890     
   + Partials       5817     3114     -2703     
   ```
   
   | [Flag](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flags&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [custom-integration1](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `<0.01% <0.00%> (ø)` | |
   | [integration](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `<0.01% <0.00%> (ø)` | |
   | [integration1](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `<0.01% <0.00%> (ø)` | |
   | [integration2](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `0.00% <0.00%> (ø)` | |
   | [java-11](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `<0.01% <0.00%> (-61.58%)` | :arrow_down: |
   | [java-21](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `34.94% <0.00%> (-26.56%)` | :arrow_down: |
   | [skip-bytebuffers-false](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `34.91% <0.00%> (-26.69%)` | :arrow_down: |
   | [skip-bytebuffers-true](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `34.92% <0.00%> (-26.57%)` | :arrow_down: |
   | [temurin](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `34.94% <0.00%> (-26.69%)` | :arrow_down: |
   | [unittests](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.78% <0.00%> (-14.84%)` | :arrow_down: |
   | [unittests1](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.78% <0.00%> (-0.19%)` | :arrow_down: |
   | [unittests2](https://app.codecov.io/gh/apache/pinot/pull/12032/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   
   </details>
   
   [:umbrella: View full report in Codecov by Sentry](https://app.codecov.io/gh/apache/pinot/pull/12032?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).   
   :loudspeaker: Have feedback on the report? [Share it here](https://about.codecov.io/codecov-pr-comment-feedback/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "aishikbh (via GitHub)" <gi...@apache.org>.
aishikbh commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1403637719


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {

Review Comment:
   Makes sense. I have made the change so that `SpecialValueTransformer` puts `null` for single valued columns for `NaN` and later `NullValueTransformer` places proper default null value later.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1401283841


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;
+    } else if ((value instanceof Double) && ((Double) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean isNoOp() {
+    return _specialValuesKeySet.isEmpty();
+  }
+
+  @Override
+  public GenericRow transform(GenericRow record) {
+    for (String element : _specialValuesKeySet) {
+      Object value = record.getValue(element);
+      if (value instanceof Float || value instanceof Double) {
+        // Single-valued column.
+        Object zeroTransformedValue = transformNegativeZero(value);
+        Object nanTransformedValue = transformNaN(zeroTransformedValue);
+        if (nanTransformedValue != value) {
+          record.putValue(element, nanTransformedValue);
+        }
+      } else if (value instanceof Object[]) {
+        // Multi-valued column.

Review Comment:
   We can simply remove it. `null` values within MV entry are ignored



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "aishikbh (via GitHub)" <gi...@apache.org>.
aishikbh commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1402046515


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/CompositeTransformer.java:
##########
@@ -68,13 +68,18 @@ public class CompositeTransformer implements RecordTransformer {
    *     Optional {@link SanitizationTransformer} after {@link NullValueTransformer} so that before sanitation, all
    *     values are non-null and follow the data types defined in the schema
    *   </li>
+   *   <li>
+   *     {@link SpecialValueTransformer} after {@link DataTypeTransformer} so that we already have the values complying
+   *      with the schema before handling special values
+   *   </li>
    * </ul>
    */
   public static List<RecordTransformer> getDefaultTransformers(TableConfig tableConfig, Schema schema) {
     return Stream.of(new ExpressionTransformer(tableConfig, schema), new FilterTransformer(tableConfig),
-        new SchemaConformingTransformer(tableConfig, schema), new DataTypeTransformer(tableConfig, schema),
-        new TimeValidationTransformer(tableConfig, schema), new NullValueTransformer(tableConfig, schema),
-        new SanitizationTransformer(schema)).filter(t -> !t.isNoOp()).collect(Collectors.toList());
+            new SchemaConformingTransformer(tableConfig, schema), new DataTypeTransformer(tableConfig, schema),

Review Comment:
   Makes sense. I have added a test to check and ensure the order of transformers.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "snleee (via GitHub)" <gi...@apache.org>.
snleee commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1401274397


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;

Review Comment:
   +1 on Jackie's comment to set this value to null instead of we explicitly try to assign NULL value in this transformer.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;
+    } else if ((value instanceof Double) && ((Double) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean isNoOp() {
+    return _specialValuesKeySet.isEmpty();
+  }
+
+  @Override
+  public GenericRow transform(GenericRow record) {
+    for (String element : _specialValuesKeySet) {
+      Object value = record.getValue(element);
+      if (value instanceof Float || value instanceof Double) {
+        // Single-valued column.
+        Object zeroTransformedValue = transformNegativeZero(value);
+        Object nanTransformedValue = transformNaN(zeroTransformedValue);
+        if (nanTransformedValue != value) {
+          record.putValue(element, nanTransformedValue);
+        }
+      } else if (value instanceof Object[]) {
+        // Multi-valued column.

Review Comment:
   Currently, our null value handler only takes care of the case when the entire value for MV column is null. 
   
   @Jackie-Jiang Do you have any recommendation on the case where one of the values in the mv column is `NaN`? Also, do we allow `null` for mv column value?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/CompositeTransformer.java:
##########
@@ -68,13 +68,18 @@ public class CompositeTransformer implements RecordTransformer {
    *     Optional {@link SanitizationTransformer} after {@link NullValueTransformer} so that before sanitation, all
    *     values are non-null and follow the data types defined in the schema
    *   </li>
+   *   <li>
+   *     {@link SpecialValueTransformer} after {@link DataTypeTransformer} so that we already have the values complying
+   *      with the schema before handling special values
+   *   </li>
    * </ul>
    */
   public static List<RecordTransformer> getDefaultTransformers(TableConfig tableConfig, Schema schema) {
     return Stream.of(new ExpressionTransformer(tableConfig, schema), new FilterTransformer(tableConfig),
-        new SchemaConformingTransformer(tableConfig, schema), new DataTypeTransformer(tableConfig, schema),
-        new TimeValidationTransformer(tableConfig, schema), new NullValueTransformer(tableConfig, schema),
-        new SanitizationTransformer(schema)).filter(t -> !t.isNoOp()).collect(Collectors.toList());
+            new SchemaConformingTransformer(tableConfig, schema), new DataTypeTransformer(tableConfig, schema),

Review Comment:
   +1 on adding the unit test for order of the transformer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "snleee (via GitHub)" <gi...@apache.org>.
snleee commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1402412167


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;
+    } else if ((value instanceof Double) && ((Double) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean isNoOp() {
+    return _specialValuesKeySet.isEmpty();
+  }
+
+  @Override
+  public GenericRow transform(GenericRow record) {
+    for (String element : _specialValuesKeySet) {
+      Object value = record.getValue(element);
+      if (value instanceof Float || value instanceof Double) {
+        // Single-valued column.
+        Object zeroTransformedValue = transformNegativeZero(value);
+        Object nanTransformedValue = transformNaN(zeroTransformedValue);
+        if (nanTransformedValue != value) {
+          record.putValue(element, nanTransformedValue);
+        }
+      } else if (value instanceof Object[]) {
+        // Multi-valued column.

Review Comment:
   Oh I see. all the operation is happening within the transformer. In that case, we can remove `NaN` from the list directly in this transformer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "snleee (via GitHub)" <gi...@apache.org>.
snleee commented on PR #12032:
URL: https://github.com/apache/pinot/pull/12032#issuecomment-1821793769

   @aishikbh Can we add the release note section on this change? The implication is that we will now start to convert the value for `NaN / -0.0`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "snleee (via GitHub)" <gi...@apache.org>.
snleee commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1401562832


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;
+    } else if ((value instanceof Double) && ((Double) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean isNoOp() {
+    return _specialValuesKeySet.isEmpty();
+  }
+
+  @Override
+  public GenericRow transform(GenericRow record) {
+    for (String element : _specialValuesKeySet) {
+      Object value = record.getValue(element);
+      if (value instanceof Float || value instanceof Double) {
+        // Single-valued column.
+        Object zeroTransformedValue = transformNegativeZero(value);
+        Object nanTransformedValue = transformNaN(zeroTransformedValue);
+        if (nanTransformedValue != value) {
+          record.putValue(element, nanTransformedValue);
+        }
+      } else if (value instanceof Object[]) {
+        // Multi-valued column.

Review Comment:
   We validated that changing value to `null` will get automatically handled by the generation layer (we do ignore null values in MV column). We can keep the current code here.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;
+    } else if ((value instanceof Double) && ((Double) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean isNoOp() {
+    return _specialValuesKeySet.isEmpty();
+  }
+
+  @Override
+  public GenericRow transform(GenericRow record) {
+    for (String element : _specialValuesKeySet) {
+      Object value = record.getValue(element);
+      if (value instanceof Float || value instanceof Double) {
+        // Single-valued column.
+        Object zeroTransformedValue = transformNegativeZero(value);
+        Object nanTransformedValue = transformNaN(zeroTransformedValue);
+        if (nanTransformedValue != value) {
+          record.putValue(element, nanTransformedValue);
+        }
+      } else if (value instanceof Object[]) {
+        // Multi-valued column.

Review Comment:
   We validated that changing value to `null` will get automatically handled by the generation layer (we do ignore null values in MV column). We can keep the current code here.
   
   cc: @aishikbh 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "aishikbh (via GitHub)" <gi...@apache.org>.
aishikbh commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1402165509


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;
+    } else if ((value instanceof Double) && ((Double) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean isNoOp() {
+    return _specialValuesKeySet.isEmpty();
+  }
+
+  @Override
+  public GenericRow transform(GenericRow record) {
+    for (String element : _specialValuesKeySet) {
+      Object value = record.getValue(element);
+      if (value instanceof Float || value instanceof Double) {
+        // Single-valued column.
+        Object zeroTransformedValue = transformNegativeZero(value);
+        Object nanTransformedValue = transformNaN(zeroTransformedValue);
+        if (nanTransformedValue != value) {
+          record.putValue(element, nanTransformedValue);
+        }
+      } else if (value instanceof Object[]) {
+        // Multi-valued column.

Review Comment:
   Seems like `null` values are removed from mv columns in `DataTypeTransformer` [here](https://github.com/apache/pinot/blob/e4d3a8dde8df8be8fcc3ae20c96ec6d004dd7933/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/DataTypeTransformer.java#L153-L157) and not by `NullValueTransformer`. So, if I put `null` instead of `NaN` in a mv column and put `SpecialValueTransformer` before `NullValueTransformer`, we end up with an unchecked `null` in the data.
   
   For single value column  replacing `NaN` with `null` and placing `SpecialValueTrasformer` before `NullValueTransformer` is reasonable as NullValueTransformer replaces `null` with default null.
   
   For mv column with `NaN`, I can think of two solutions:
   1. Replace `null` with default null in `SpecialValueTransformer`
   2. Remove NaN from the list of values (as we do with `null`)
   
   I think solution 1. should be reasonable as we are replacing `NaN` in single valued column with default null anyway.
   
   Thoughts @Jackie-Jiang @snleee ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1399821015


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/CompositeTransformer.java:
##########
@@ -68,13 +68,18 @@ public class CompositeTransformer implements RecordTransformer {
    *     Optional {@link SanitizationTransformer} after {@link NullValueTransformer} so that before sanitation, all
    *     values are non-null and follow the data types defined in the schema
    *   </li>
+   *   <li>
+   *     {@link SpecialValueTransformer} after {@link DataTypeTransformer} so that we already have the values complying
+   *      with the schema before handling special values
+   *   </li>
    * </ul>
    */
   public static List<RecordTransformer> getDefaultTransformers(TableConfig tableConfig, Schema schema) {
     return Stream.of(new ExpressionTransformer(tableConfig, schema), new FilterTransformer(tableConfig),
-        new SchemaConformingTransformer(tableConfig, schema), new DataTypeTransformer(tableConfig, schema),
-        new TimeValidationTransformer(tableConfig, schema), new NullValueTransformer(tableConfig, schema),
-        new SanitizationTransformer(schema)).filter(t -> !t.isNoOp()).collect(Collectors.toList());
+            new SchemaConformingTransformer(tableConfig, schema), new DataTypeTransformer(tableConfig, schema),

Review Comment:
   We should add test to ensure the expected behavior. There is java doc above the method: `NOTE: DO NOT CHANGE THE ORDER OF THE RECORD TRANSFORMERS`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "swaminathanmanish (via GitHub)" <gi...@apache.org>.
swaminathanmanish commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1399771663


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/CompositeTransformer.java:
##########
@@ -68,13 +68,18 @@ public class CompositeTransformer implements RecordTransformer {
    *     Optional {@link SanitizationTransformer} after {@link NullValueTransformer} so that before sanitation, all
    *     values are non-null and follow the data types defined in the schema
    *   </li>
+   *   <li>
+   *     {@link SpecialValueTransformer} after {@link DataTypeTransformer} so that we already have the values complying
+   *      with the schema before handling special values
+   *   </li>
    * </ul>
    */
   public static List<RecordTransformer> getDefaultTransformers(TableConfig tableConfig, Schema schema) {
     return Stream.of(new ExpressionTransformer(tableConfig, schema), new FilterTransformer(tableConfig),
-        new SchemaConformingTransformer(tableConfig, schema), new DataTypeTransformer(tableConfig, schema),
-        new TimeValidationTransformer(tableConfig, schema), new NullValueTransformer(tableConfig, schema),
-        new SanitizationTransformer(schema)).filter(t -> !t.isNoOp()).collect(Collectors.toList());
+            new SchemaConformingTransformer(tableConfig, schema), new DataTypeTransformer(tableConfig, schema),

Review Comment:
   Im curious how we ensure that order of transformers is preserved @Jackie-Jiang , @snleee . If we accidentally mess the order, do we have a problem and do we catch it in our tests?
   



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;
+    } else if ((value instanceof Double) && ((Double) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean isNoOp() {
+    return _specialValuesKeySet.isEmpty();
+  }
+
+  @Override
+  public GenericRow transform(GenericRow record) {
+    for (String element : _specialValuesKeySet) {
+      Object value = record.getValue(element);
+      if (value instanceof Float || value instanceof Double) {

Review Comment:
   I guess we dont need these checks since its already done in the individual methods.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {

Review Comment:
   Is it necessary to emit any logs/metrics when this happens? Would it be useful for debugging?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "aishikbh (via GitHub)" <gi...@apache.org>.
aishikbh commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1402047228


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {

Review Comment:
   I have added logs when we are transforming the values.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "aishikbh (via GitHub)" <gi...@apache.org>.
aishikbh commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1403637719


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {

Review Comment:
   Makes sense. I have made the change so that `SpecialValueTransformer` puts `null` for single valued columns for `NaN` and later `NullValueTransformer` places proper default value later.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;

Review Comment:
   added the change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1399825053


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {

Review Comment:
   We can put this transformer after `DataTypeTransformer` but before `NullValueTransformer` so that when encountering `NaN` we can put `null` so that proper default value can be added by `NullValueTransformer`. You may refer to `TimeValidationTransformer` which performs similar value checks



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "snleee (via GitHub)" <gi...@apache.org>.
snleee merged PR #12032:
URL: https://github.com/apache/pinot/pull/12032


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "aishikbh (via GitHub)" <gi...@apache.org>.
aishikbh commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1403637719


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {

Review Comment:
   Makes sense. I have made the change so that `SpecialValueTransformer` puts `null` for single valued columns for `NaN` and later `NullValueTransformer` places proper default value.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "aishikbh (via GitHub)" <gi...@apache.org>.
aishikbh commented on PR #12032:
URL: https://github.com/apache/pinot/pull/12032#issuecomment-1824777003

   > @aishikbh Can we add the release note section on this change? The implication is that we will now start to handle values for `NaN / -0.0`
   
   Added release notes for the change


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "aishikbh (via GitHub)" <gi...@apache.org>.
aishikbh commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1403638354


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;
+    } else if ((value instanceof Double) && ((Double) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean isNoOp() {
+    return _specialValuesKeySet.isEmpty();
+  }
+
+  @Override
+  public GenericRow transform(GenericRow record) {
+    for (String element : _specialValuesKeySet) {
+      Object value = record.getValue(element);
+      if (value instanceof Float || value instanceof Double) {
+        // Single-valued column.
+        Object zeroTransformedValue = transformNegativeZero(value);
+        Object nanTransformedValue = transformNaN(zeroTransformedValue);
+        if (nanTransformedValue != value) {
+          record.putValue(element, nanTransformedValue);
+        }
+      } else if (value instanceof Object[]) {
+        // Multi-valued column.

Review Comment:
   made the change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "aishikbh (via GitHub)" <gi...@apache.org>.
aishikbh commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1402048231


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;
+    } else if ((value instanceof Double) && ((Double) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean isNoOp() {
+    return _specialValuesKeySet.isEmpty();
+  }
+
+  @Override
+  public GenericRow transform(GenericRow record) {
+    for (String element : _specialValuesKeySet) {
+      Object value = record.getValue(element);
+      if (value instanceof Float || value instanceof Double) {

Review Comment:
   Makes sense. Removed the check here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "aishikbh (via GitHub)" <gi...@apache.org>.
aishikbh commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1402165509


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;
+    } else if ((value instanceof Double) && ((Double) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean isNoOp() {
+    return _specialValuesKeySet.isEmpty();
+  }
+
+  @Override
+  public GenericRow transform(GenericRow record) {
+    for (String element : _specialValuesKeySet) {
+      Object value = record.getValue(element);
+      if (value instanceof Float || value instanceof Double) {
+        // Single-valued column.
+        Object zeroTransformedValue = transformNegativeZero(value);
+        Object nanTransformedValue = transformNaN(zeroTransformedValue);
+        if (nanTransformedValue != value) {
+          record.putValue(element, nanTransformedValue);
+        }
+      } else if (value instanceof Object[]) {
+        // Multi-valued column.

Review Comment:
   Seems like `null` values are removed from mv columns in `DataTypeTransformer` [here](https://github.com/apache/pinot/blob/e4d3a8dde8df8be8fcc3ae20c96ec6d004dd7933/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/DataTypeTransformer.java#L153-L157) and not by `NullValueTransformer`. So, if I put `null` instead of `NaN` in a mv column, we end up with an unchecked `null` in the data.
   
   For single value column  replacing `NaN` with `null` and placing `SpecialValueTrasformer` before `NullValueTransformer` is reasonable as NullValueTransformer replaces `null` with default null.
   
   For mv column with `NaN`, I can think of two solutions:
   1. Replace `null` with default null in `SpecialValueTransformer`
   2. Remove NaN from the list of values (as we do with `null`)
   
   I think solution 1. should be reasonable as we are replacing `NaN` in single valued column with default null anyway.
   
   Thoughts @Jackie-Jiang @snleee ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] [bugfix] Add new Transformer to transform -0.0 and NaN [pinot]

Posted by "aishikbh (via GitHub)" <gi...@apache.org>.
aishikbh commented on code in PR #12032:
URL: https://github.com/apache/pinot/pull/12032#discussion_r1402165509


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SpecialValueTransformer.java:
##########
@@ -0,0 +1,96 @@
+/**
+ * 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.pinot.segment.local.recordtransformer;
+
+import java.util.HashSet;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+/**
+ * The {@code SpecialValueTransformer} class will transform special values according to the following rules:
+ * <ul>
+ *   <li>Negative zero (-0.0) should be converted to 0.0</li>
+ *   <li>NaN should be converted to default null</li>
+ * </ul>
+ * <p>NOTE: should put this after the {@link DataTypeTransformer} so that all values follow the data types in
+ * {@link FieldSpec}.
+ */
+public class SpecialValueTransformer implements RecordTransformer {
+  private final HashSet<String> _specialValuesKeySet = new HashSet<>();
+
+  public SpecialValueTransformer(Schema schema) {
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn() && (fieldSpec.getDataType() == DataType.FLOAT
+          || fieldSpec.getDataType() == DataType.DOUBLE)) {
+        _specialValuesKeySet.add(fieldSpec.getName());
+      }
+    }
+  }
+
+  private Object transformNegativeZero(Object value) {
+    if ((value instanceof Float) && (Float.floatToRawIntBits((float) value) == Float.floatToRawIntBits(-0.0f))) {
+      value = 0.0f;
+    } else if ((value instanceof Double) && (Double.doubleToLongBits((double) value) == Double.doubleToLongBits(
+        -0.0d))) {
+      value = 0.0d;
+    }
+    return value;
+  }
+
+  private Object transformNaN(Object value) {
+    if ((value instanceof Float) && ((Float) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT;
+    } else if ((value instanceof Double) && ((Double) value).isNaN()) {
+      value = FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE;
+    }
+    return value;
+  }
+
+  @Override
+  public boolean isNoOp() {
+    return _specialValuesKeySet.isEmpty();
+  }
+
+  @Override
+  public GenericRow transform(GenericRow record) {
+    for (String element : _specialValuesKeySet) {
+      Object value = record.getValue(element);
+      if (value instanceof Float || value instanceof Double) {
+        // Single-valued column.
+        Object zeroTransformedValue = transformNegativeZero(value);
+        Object nanTransformedValue = transformNaN(zeroTransformedValue);
+        if (nanTransformedValue != value) {
+          record.putValue(element, nanTransformedValue);
+        }
+      } else if (value instanceof Object[]) {
+        // Multi-valued column.

Review Comment:
   Seems like `null` values are removed from mv columns in `DataTypeTransformer` [here](https://github.com/apache/pinot/blob/e4d3a8dde8df8be8fcc3ae20c96ec6d004dd7933/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/DataTypeTransformer.java#L153-L157) and not by `NullValueTransformer`. So, if I put `null` instead of `NaN` in a mv column, we end up with an unchecked `null` in the data.
   
   For single value column  replacing `NaN` with `null` is reasonable as NullValueTransformer replaces `null` with default null.
   
   For mv column with `NaN`, I can think of two solutions:
   1. Replace `null` with default null in `SpecialValueTransformer`
   2. Remove NaN from the list of values (as we do with `null`)
   
   I think solution 1. should be reasonable as we are replacing `NaN` in single valued column with default null anyway.
   
   Thoughts @Jackie-Jiang @snleee ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org