You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "lsyldliu (via GitHub)" <gi...@apache.org> on 2023/03/03 02:15:49 UTC

[GitHub] [flink] lsyldliu commented on a diff in pull request #22031: [FLINK-31239][hive] Fix native sum function can't get the corrected value when the argument type is string

lsyldliu commented on code in PR #22031:
URL: https://github.com/apache/flink/pull/22031#discussion_r1123961288


##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java:
##########
@@ -88,27 +97,27 @@ public Expression[] retractExpressions() {
 
     @Override
     public Expression[] mergeExpressions() {
+        Expression coalesceSum = coalesce(sum, zero);
         return new Expression[] {
             /* sum = */ ifThenElse(
                     isNull(mergeOperand(sum)),
-                    sum,
-                    ifThenElse(
-                            isNull(sum),
-                            mergeOperand(sum),
-                            adjustedPlus(getResultType(), sum, mergeOperand(sum))))
+                    coalesceSum,
+                    adjustedPlus(getResultType(), coalesceSum, mergeOperand(sum))),
+            and(isEmpty, mergeOperand(isEmpty))
         };
     }
 
     @Override
     public Expression getValueExpression() {
-        return sum;
+        return ifThenElse(isTrue(isEmpty), nullOf(getResultType()), sum);

Review Comment:
   Hive code as following: 
   ```
   @AggregationType(estimable = true)
       static class SumLongAgg extends SumAgg<Long> {
         @Override
         public int estimate() { return JavaDataModel.PRIMITIVES1 + JavaDataModel.PRIMITIVES2; }
       }
   
       @Override
       public AggregationBuffer getNewAggregationBuffer() throws HiveException {
         SumLongAgg result = new SumLongAgg();
         reset(result);
         return result;
       }
   
       @Override
       public void reset(AggregationBuffer agg) throws HiveException {
         SumLongAgg myagg = (SumLongAgg) agg;
         myagg.empty = true;
         myagg.sum = 0L;
         myagg.uniqueObjects = new HashSet<ObjectInspectorObject>();
       }
   
       private boolean warned = false;
   
       @Override
       public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
         assert (parameters.length == 1);
         try {
           if (isEligibleValue((SumLongAgg) agg, parameters[0])) {
             ((SumLongAgg)agg).empty = false;
             ((SumLongAgg)agg).sum += PrimitiveObjectInspectorUtils.getLong(parameters[0], inputOI);
           }
         } catch (NumberFormatException e) {
           if (!warned) {
             warned = true;
             LOG.warn(getClass().getSimpleName() + " "
                 + StringUtils.stringifyException(e));
           }
         }
       }
   
       @Override
       public void merge(AggregationBuffer agg, Object partial) throws HiveException {
         if (partial != null) {
           SumLongAgg myagg = (SumLongAgg) agg;
           myagg.empty = false;
           if (isWindowingDistinct()) {
             throw new HiveException("Distinct windowing UDAF doesn't support merge and terminatePartial");
           } else {
               myagg.sum += PrimitiveObjectInspectorUtils.getLong(partial, inputOI);
           }
         }
       }
   
       @Override
       public Object terminate(AggregationBuffer agg) throws HiveException {
         SumLongAgg myagg = (SumLongAgg) agg;
         if (myagg.empty) {
           return null;
         }
         result.set(myagg.sum);
         return result;
       }
   ```
   It returns a null value if all elements are null.



-- 
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: issues-unsubscribe@flink.apache.org

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