You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by lv...@apache.org on 2019/06/10 16:18:38 UTC

[impala] 04/05: IMPALA-5031: signed overflow is undefined behavior

This is an automated email from the ASF dual-hosted git repository.

lv pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 45c6c46bf6d051c715fe75a68c7970b208a7d376
Author: Jim Apple <jb...@apache.org>
AuthorDate: Sat May 25 17:50:21 2019 -0700

    IMPALA-5031: signed overflow is undefined behavior
    
    Fix remaining signed overflow undefined behaviors in end-to-end
    tests. The interesting part of the backtraces:
    
        exprs/aggregate-functions-ir.cc:464:25: runtime error: signed
           integer overflow: 0x5a4728ca063b522c0b728f8000000000 +
           0x3c2f7086aed236c807a1b50000000000 cannot be represented in
           type '__int128'
        #0 AggregateFunctions::DecimalAvgMerge(
           impala_udf::FunctionContext*, impala_udf::StringVal const&,
           impala_udf::StringVal*) exprs/aggregate-functions-ir.cc:464:25
        #1 AggFnEvaluator::Update(TupleRow const*, Tuple*, void*)
           exprs/agg-fn-evaluator.cc:327:7
        #2 AggFnEvaluator::Add(TupleRow const*, Tuple*)
           exprs/agg-fn-evaluator.h:257:3
        #3 Aggregator::UpdateTuple(AggFnEvaluator**, Tuple*, TupleRow*, bool)
           exec/aggregator.cc:167:24
        #4 NonGroupingAggregator::AddBatchImpl(RowBatch*)
           exec/non-grouping-aggregator-ir.cc:27:5
        #5 NonGroupingAggregator::AddBatch(RuntimeState*, RowBatch*)
           exec/non-grouping-aggregator.cc:124:45
        #6 AggregationNode::Open(RuntimeState*)
           exec/aggregation-node.cc:70:57
    
        exprs/aggregate-functions-ir.cc:513:12: runtime error: signed
           integer overflow: -8282081183197145958 + -4473782455107795527
           cannot be represented in type 'long'
        #0 void AggregateFunctions::SumUpdate<impala_udf::BigIntVal,
           impala_udf::BigIntVal>(impala_udf::FunctionContext*,
           impala_udf::BigIntVal const&, impala_udf::BigIntVal*)
           exprs/aggregate-functions-ir.cc:513:12
        #1 AggFnEvaluator::Update(TupleRow const*, Tuple*, void*)
           exprs/agg-fn-evaluator.cc:327:7
        #2 AggFnEvaluator::Add(TupleRow const*, Tuple*)
           exprs/agg-fn-evaluator.h:257:3
        #3 Aggregator::UpdateTuple(AggFnEvaluator**, Tuple*, TupleRow*,
           bool) exec/aggregator.cc:167:24
        #4 NonGroupingAggregator::AddBatchImpl(RowBatch*)
           exec/non-grouping-aggregator-ir.cc:27:5
        #5 NonGroupingAggregator::AddBatch(RuntimeState*, RowBatch*)
           exec/non-grouping-aggregator.cc:124:45
        #6 AggregationNode::Open(RuntimeState*)
           exec/aggregation-node.cc:70:57
    
        exprs/aggregate-functions-ir.cc:585:14: runtime error: signed
           integer overflow: 0x5a4728ca063b522c0b728f8000000000 +
           0x3c2f7086aed236c807a1b50000000000 cannot be represented in
           type '__int128'
        #0 AggregateFunctions::SumDecimalMerge(
           impala_udf::FunctionContext*, impala_udf::DecimalVal const&,
           impala_udf::DecimalVal*) exprs/aggregate-functions-ir.cc:585:14
        #1 AggFnEvaluator::Update(TupleRow const*, Tuple*, void*)
           exprs/agg-fn-evaluator.cc:327:7
        #2 AggFnEvaluator::Add(TupleRow const*, Tuple*)
           exprs/agg-fn-evaluator.h:257:3
        #3 Aggregator::UpdateTuple(AggFnEvaluator**, Tuple*, TupleRow*, bool)
           exec/aggregator.cc:167:24
        #4 NonGroupingAggregator::AddBatchImpl(RowBatch*)
           exec/non-grouping-aggregator-ir.cc:27:5
        #5 NonGroupingAggregator::AddBatch(RuntimeState*, RowBatch*)
           exec/non-grouping-aggregator.cc:124:45
        #6 AggregationNode::Open(RuntimeState*)
           exec/aggregation-node.cc:70:57
    
        runtime/decimal-value.inline.h:145:12: runtime error: signed
           integer overflow: 18 * 0x0785ee10d5da46d900f436a000000000 cannot
           be represented in type '__int128'
        #0 DecimalValue<__int128>::ScaleTo(int, int, int, bool*) const
           runtime/decimal-value.inline.h:145:12
        #1 DecimalOperators::ScaleDecimalValue(
          impala_udf::FunctionContext*, DecimalValue<int> const&, int,
          int, int) exprs/decimal-operators-ir.cc:132:41
        #2 DecimalOperators::RoundDecimal(impala_udf::FunctionContext*,
           impala_udf::DecimalVal const&, int, int, int, int,
           DecimalOperators::DecimalRoundOp const&)
           exprs/decimal-operators-ir.cc:465:16
        #3 DecimalOperators::RoundDecimal(impala_udf::FunctionContext*,
           impala_udf::DecimalVal const&, DecimalOperators::DecimalRoundOp
           const&) exprs/decimal-operators-ir.cc:519:10
        #4 DecimalOperators::CastToDecimalVal(
           impala_udf::FunctionContext*, impala_udf::DecimalVal const&)
           exprs/decimal-operators-ir.cc:529:10
        #5 impala_udf::DecimalVal ScalarFnCall::InterpretEval
           <impala_udf::DecimalVal>(ScalarExprEvaluator*, TupleRow const*)
           const exprs/scalar-fn-call.cc:485:208
        #6 ScalarFnCall::GetDecimalVal(ScalarExprEvaluator*, TupleRow
           const*) const exprs/scalar-fn-call.cc:618:44
        #7 ScalarExprEvaluator::GetValue(ScalarExpr const&, TupleRow
           const*) exprs/scalar-expr-evaluator.cc:321:27
        #8 ScalarExprEvaluator::GetValue(TupleRow const*)
           exprs/scalar-expr-evaluator.cc:251:10
        #9 Java_org_apache_impala_service_FeSupport_NativeEvalExprsWithoutRow
           service/fe-support.cc:246:26
        #10 (<unknown module>)
    
        runtime/multi-precision.h:116:21: runtime error: negation of
           0x80000000000000000000000000000000 cannot be represented in
           type 'int128_t' (aka '__int128'); cast to an unsigned type to
           negate this value to itself
        #0 ConvertToInt128(boost::multiprecision::number
           <boost::multiprecision::backends::cpp_int_backend<256u, 256u,
           (boost::multiprecision::cpp_integer_type)1,
           (boost::multiprecision::cpp_int_check_type)0, void>,
           (boost::multiprecision::expression_template_option)0>,
           __int128, bool*) runtime/multi-precision.h:116:21
        #1 DecimalValue<__int128>
           DecimalValue<__int128>::Multiply<__int128>(int,
           DecimalValue<__int128> const&, int, int, int, bool, bool*) const
           runtime/decimal-value.inline.h:438:16
        #2 DecimalOperators::Multiply_DecimalVal_DecimalVal(
           impala_udf::FunctionContext*, impala_udf::DecimalVal const&,
           impala_udf::DecimalVal const&)
           exprs/decimal-operators-ir.cc:859:3336
        #3 impala_udf::DecimalVal ScalarFnCall::InterpretEval
           <impala_udf::DecimalVal>(ScalarExprEvaluator*, TupleRow const*)
           const exprs/scalar-fn-call.cc:485:376
        #4 ScalarFnCall::GetDecimalVal(ScalarExprEvaluator*, TupleRow
           const*) const exprs/scalar-fn-call.cc:618:44
        #5 ScalarExprEvaluator::GetValue(ScalarExpr const&, TupleRow
           const*) exprs/scalar-expr-evaluator.cc:321:27
        #6 ScalarExprEvaluator::GetValue(TupleRow const*)
           exprs/scalar-expr-evaluator.cc:251:10
        #7 Java_org_apache_impala_service_FeSupport_NativeEvalExprsWithoutRow
           service/fe-support.cc:246:26
        #8 (<unknown module>)
    
        util/runtime-profile-counters.h:194:24: runtime error: signed
           integer overflow: -1263418397011577524 + -9223370798768111350
           cannot be represented in type 'long'
        #0 RuntimeProfile::AveragedCounter::UpdateCounter
           (RuntimeProfile::Counter*)
           util/runtime-profile-counters.h:194:24
        #1 RuntimeProfile::UpdateAverage(RuntimeProfile*)
           util/runtime-profile.cc:199:20
        #2 RuntimeProfile::UpdateAverage(RuntimeProfile*)
           util/runtime-profile.cc:245:14
        #3 Coordinator::BackendState::UpdateExecStats
           (vector<Coordinator::FragmentStats*,
           allocator<Coordinator::FragmentStats*> > const&)
           runtime/coordinator-backend-state.cc:429:22
        #4 Coordinator::ComputeQuerySummary()
           runtime/coordinator.cc:775:20
        #5 Coordinator::HandleExecStateTransition(Coordinator::ExecState,
           Coordinator::ExecState) runtime/coordinator.cc:567:3
        #6 Coordinator::SetNonErrorTerminalState(Coordinator::ExecState)
           runtime/coordinator.cc:484:3
        #7 Coordinator::GetNext(QueryResultSet*, int, bool*)
           runtime/coordinator.cc:657:53
        #8 ClientRequestState::FetchRowsInternal(int, QueryResultSet*)
           service/client-request-state.cc:943:34
        #9 ClientRequestState::FetchRows(int, QueryResultSet*)
           service/client-request-state.cc:835:36
        #10 ImpalaServer::FetchInternal(TUniqueId const&, bool, int,
            beeswax::Results*) service/impala-beeswax-server.cc:545:40
        #11 ImpalaServer::fetch(beeswax::Results&, beeswax::QueryHandle
            const&, bool, int) service/impala-beeswax-server.cc:178:19
        #12 beeswax::BeeswaxServiceProcessor::process_fetch(int,
            apache::thrift::protocol::TProtocol*,
            apache::thrift::protocol::TProtocol*, void*)
            generated-sources/gen-cpp/BeeswaxService.cpp:3398:13
        #13 beeswax::BeeswaxServiceProcessor::dispatchCall
            (apache::thrift::protocol::TProtocol*,
            apache::thrift::protocol::TProtocol*, string const&, int,
            void*) generated-sources/gen-cpp/BeeswaxService.cpp:3200:3
        #14 ImpalaServiceProcessor::dispatchCall
            (apache::thrift::protocol::TProtocol*,
            apache::thrift::protocol::TProtocol*, string const&, int,
            void*) generated-sources/gen-cpp/ImpalaService.cpp:1824:48
        #15 apache::thrift::TDispatchProcessor::process
            (boost::shared_ptr<apache::thrift::protocol::TProtocol>,
            boost::shared_ptr<apache::thrift::protocol::TProtocol>, void*)
            toolchain/thrift-0.9.3-p5/include/thrift/TDispatchProcessor.h:121:12
    
    Change-Id: I73dd6802ec1023275d09a99a2950f3558313fc8e
    Reviewed-on: http://gerrit.cloudera.org:8080/13437
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exprs/aggregate-functions-ir.cc | 8 +++++---
 be/src/runtime/decimal-value.inline.h  | 2 +-
 be/src/runtime/multi-precision.h       | 2 +-
 be/src/util/arithmetic-util.h          | 9 +++++++++
 be/src/util/runtime-profile-counters.h | 4 +++-
 5 files changed, 19 insertions(+), 6 deletions(-)

diff --git a/be/src/exprs/aggregate-functions-ir.cc b/be/src/exprs/aggregate-functions-ir.cc
index e584cca..1c9bd85 100644
--- a/be/src/exprs/aggregate-functions-ir.cc
+++ b/be/src/exprs/aggregate-functions-ir.cc
@@ -36,6 +36,7 @@
 #include "runtime/string-value.inline.h"
 #include "runtime/timestamp-value.h"
 #include "runtime/timestamp-value.inline.h"
+#include "util/arithmetic-util.h"
 #include "util/mpfit-util.h"
 
 #include "common/names.h"
@@ -461,7 +462,8 @@ void AggregateFunctions::DecimalAvgMerge(FunctionContext* ctx,
       abs(dst_struct->sum_val16) >
       DecimalUtil::MAX_UNSCALED_DECIMAL16 - abs(src_struct->sum_val16);
   if (UNLIKELY(overflow)) ctx->SetError("Avg computation overflowed");
-  dst_struct->sum_val16 += src_struct->sum_val16;
+  dst_struct->sum_val16 =
+      ArithmeticUtil::AsUnsigned<std::plus>(dst_struct->sum_val16, src_struct->sum_val16);
   dst_struct->count += src_struct->count;
 }
 
@@ -510,7 +512,7 @@ void AggregateFunctions::SumUpdate(FunctionContext* ctx, const SRC_VAL& src,
     return;
   }
   if (dst->is_null) InitZero<DST_VAL>(ctx, dst);
-  dst->val += src.val;
+  dst->val = ArithmeticUtil::Compute<std::plus, decltype(dst->val)>(dst->val, src.val);
 }
 
 template<typename SRC_VAL, typename DST_VAL>
@@ -582,7 +584,7 @@ void AggregateFunctions::SumDecimalMerge(FunctionContext* ctx,
   bool overflow = decimal_v2 &&
       abs(dst->val16) > DecimalUtil::MAX_UNSCALED_DECIMAL16 - abs(src.val16);
   if (UNLIKELY(overflow)) ctx->SetError("Sum computation overflowed");
-  dst->val16 += src.val16;
+  dst->val16 = ArithmeticUtil::AsUnsigned<std::plus>(dst->val16, src.val16);
 }
 
 template<typename T>
diff --git a/be/src/runtime/decimal-value.inline.h b/be/src/runtime/decimal-value.inline.h
index 0a4866e..6480099 100644
--- a/be/src/runtime/decimal-value.inline.h
+++ b/be/src/runtime/decimal-value.inline.h
@@ -142,7 +142,7 @@ inline DecimalValue<T> DecimalValue<T>::ScaleTo(int src_scale, int dst_scale,
   } else if (delta_scale < 0) {
     T mult = DecimalUtil::GetScaleMultiplier<T>(-delta_scale);
     *overflow |= abs(result) >= max_value / mult;
-    result *= mult;
+    result = ArithmeticUtil::AsUnsigned<std::multiplies>(result, mult);
   }
   return DecimalValue(result);
 }
diff --git a/be/src/runtime/multi-precision.h b/be/src/runtime/multi-precision.h
index 0d8028f..5abc6a3 100644
--- a/be/src/runtime/multi-precision.h
+++ b/be/src/runtime/multi-precision.h
@@ -113,7 +113,7 @@ inline int128_t ConvertToInt128(int256_t x, int128_t max_value, bool* overflow)
     scale =
         ArithmeticUtil::AsUnsigned<std::multiplies>(scale, static_cast<int128_t>(base));
   }
-  return negative ? -result : result;
+  return negative ? ArithmeticUtil::Negate(result) : result;
 }
 
 /// abs() is not defined for int128_t. Name it abs() so it can be compatible with
diff --git a/be/src/util/arithmetic-util.h b/be/src/util/arithmetic-util.h
index 4ab60cd..d6a6446 100644
--- a/be/src/util/arithmetic-util.h
+++ b/be/src/util/arithmetic-util.h
@@ -113,6 +113,15 @@ class ArithmeticUtil {
     return OperateOn<T>::template Compute<Operator>(x, y);
   }
 
+  // Negation of the least value of signed two's-complement types is undefined behavior.
+  // This operator makes that behavior defined by doing it in the unsigned domain. Note
+  // that this induces Negate(INT_MIN) == INT_MIN, though otherwise produces identical
+  // behavior to just using the usual unary negation operator like "-x".
+  template<typename T>
+  static T Negate(T x) {
+    return ToSigned(-ToUnsigned(x));
+  }
+
  private:
   // Ring and OperateOn are used for compile-time dispatching on how Compute() should
   // perform an arithmetic operation: as an unsigned integer operation, as a
diff --git a/be/src/util/runtime-profile-counters.h b/be/src/util/runtime-profile-counters.h
index a9b3875..cb557ce 100644
--- a/be/src/util/runtime-profile-counters.h
+++ b/be/src/util/runtime-profile-counters.h
@@ -26,6 +26,7 @@
 
 #include "common/atomic.h"
 #include "common/logging.h"
+#include "util/arithmetic-util.h"
 #include "util/runtime-profile.h"
 #include "util/stopwatch.h"
 #include "util/streaming-sampler.h"
@@ -191,7 +192,8 @@ class RuntimeProfile::AveragedCounter : public RuntimeProfile::Counter {
       double result_val = current_double_sum_ / (double) counter_value_map_.size();
       value_.Store(*reinterpret_cast<int64_t*>(&result_val));
     } else {
-      current_int_sum_ += (new_counter->value() - old_val);
+      current_int_sum_ = ArithmeticUtil::AsUnsigned<std::plus>(
+          current_int_sum_, (new_counter->value() - old_val));
       value_.Store(current_int_sum_ / counter_value_map_.size());
     }
   }