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

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

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