You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ra...@apache.org on 2019/06/12 08:45:33 UTC

[arrow] branch master updated: ARROW-5451: [C++][Gandiva] Support cast/round functions for decimal

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 01dcb45  ARROW-5451: [C++][Gandiva] Support cast/round functions for decimal
01dcb45 is described below

commit 01dcb455c9e89854625322a1f053468f430a039b
Author: Pindikura Ravindra <ra...@dremio.com>
AuthorDate: Wed Jun 12 14:15:15 2019 +0530

    ARROW-5451: [C++][Gandiva] Support cast/round functions for decimal
    
    Author: Pindikura Ravindra <ra...@dremio.com>
    
    Closes #4418 from pravindra/cast and squashes the following commits:
    
    1f499dfb <Pindikura Ravindra> ARROW-5451:  Support cast/round functions for decimal
---
 cpp/src/gandiva/decimal_ir.cc                   | 143 +++++++
 cpp/src/gandiva/decimal_ir.h                    |   3 +
 cpp/src/gandiva/function_registry_arithmetic.cc |  29 +-
 cpp/src/gandiva/function_registry_math_ops.cc   |  12 +-
 cpp/src/gandiva/precompiled/decimal_ops.cc      | 239 +++++++++++
 cpp/src/gandiva/precompiled/decimal_ops.h       |  29 ++
 cpp/src/gandiva/precompiled/decimal_ops_test.cc | 502 ++++++++++++++++++++++++
 cpp/src/gandiva/precompiled/decimal_wrapper.cc  | 126 ++++++
 cpp/src/gandiva/tests/decimal_test.cc           | 185 ++++++++-
 9 files changed, 1253 insertions(+), 15 deletions(-)

diff --git a/cpp/src/gandiva/decimal_ir.cc b/cpp/src/gandiva/decimal_ir.cc
index 6344332..d291611 100644
--- a/cpp/src/gandiva/decimal_ir.cc
+++ b/cpp/src/gandiva/decimal_ir.cc
@@ -502,8 +502,63 @@ Status DecimalIR::BuildCompare(const std::string& function_name,
   return Status::OK();
 }
 
+Status DecimalIR::BuildDecimalFunction(const std::string& function_name,
+                                       llvm::Type* return_type,
+                                       std::vector<NamedArg> in_types) {
+  auto i64 = types()->i64_type();
+  auto i128 = types()->i128_type();
+  auto function = BuildFunction(function_name, return_type, in_types);
+
+  auto entry = llvm::BasicBlock::Create(*context(), "entry", function);
+  ir_builder()->SetInsertPoint(entry);
+
+  std::vector<llvm::Value*> args;
+  int arg_idx = 0;
+  auto arg_iter = function->arg_begin();
+  for (auto& type : in_types) {
+    if (type.type == i128) {
+      // split i128 arg into two int64s.
+      auto split = ValueSplit::MakeFromInt128(this, &arg_iter[arg_idx]);
+      args.push_back(split.high());
+      args.push_back(split.low());
+    } else {
+      args.push_back(&arg_iter[arg_idx]);
+    }
+    ++arg_idx;
+  }
+
+  auto internal_name = function_name + "_internal";
+  llvm::Value* result = nullptr;
+  if (return_type == i128) {
+    // for i128 ret, replace with two int64* args, and join them.
+    auto block = ir_builder()->GetInsertBlock();
+    auto out_high_ptr = new llvm::AllocaInst(i64, 0, "out_hi", block);
+    auto out_low_ptr = new llvm::AllocaInst(i64, 0, "out_low", block);
+    args.push_back(out_high_ptr);
+    args.push_back(out_low_ptr);
+
+    // Make call to pre-compiled IR function.
+    ir_builder()->CreateCall(module()->getFunction(internal_name), args);
+
+    auto out_high = ir_builder()->CreateLoad(out_high_ptr);
+    auto out_low = ir_builder()->CreateLoad(out_low_ptr);
+    result = ValueSplit(out_high, out_low).AsInt128(this);
+  } else {
+    DCHECK_NE(return_type, types()->void_type());
+
+    // Make call to pre-compiled IR function.
+    result = ir_builder()->CreateCall(module()->getFunction(internal_name), args);
+  }
+  ir_builder()->CreateRet(result);
+  return Status::OK();
+}
+
 Status DecimalIR::AddFunctions(Engine* engine) {
   auto decimal_ir = std::make_shared<DecimalIR>(engine);
+  auto i128 = decimal_ir->types()->i128_type();
+  auto i32 = decimal_ir->types()->i32_type();
+  auto i64 = decimal_ir->types()->i64_type();
+  auto f64 = decimal_ir->types()->double_type();
 
   // Populate global variables used by decimal operations.
   decimal_ir->AddGlobals(engine);
@@ -531,6 +586,94 @@ Status DecimalIR::AddFunctions(Engine* engine) {
                                                llvm::ICmpInst::ICMP_SGT));
   ARROW_RETURN_NOT_OK(decimal_ir->BuildCompare(
       "greater_than_or_equal_to_decimal128_decimal128", llvm::ICmpInst::ICMP_SGE));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("abs_decimal128", i128,
+                                                       {
+                                                           {"x_value", i128},
+                                                           {"x_precision", i32},
+                                                           {"x_scale", i32},
+                                                       }));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("ceil_decimal128", i128,
+                                                       {
+                                                           {"x_value", i128},
+                                                           {"x_precision", i32},
+                                                           {"x_scale", i32},
+                                                       }));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("floor_decimal128", i128,
+                                                       {
+                                                           {"x_value", i128},
+                                                           {"x_precision", i32},
+                                                           {"x_scale", i32},
+                                                       }));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("round_decimal128", i128,
+                                                       {
+                                                           {"x_value", i128},
+                                                           {"x_precision", i32},
+                                                           {"x_scale", i32},
+                                                       }));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("round_decimal128_int32", i128,
+                                                       {
+                                                           {"x_value", i128},
+                                                           {"x_precision", i32},
+                                                           {"x_scale", i32},
+                                                           {"round_scale", i32},
+                                                       }));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("truncate_decimal128", i128,
+                                                       {
+                                                           {"x_value", i128},
+                                                           {"x_precision", i32},
+                                                           {"x_scale", i32},
+                                                       }));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("truncate_decimal128_int32", i128,
+                                                       {
+                                                           {"x_value", i128},
+                                                           {"x_precision", i32},
+                                                           {"x_scale", i32},
+                                                           {"round_scale", i32},
+                                                       }));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("castDECIMAL_int64", i128,
+                                                       {
+                                                           {"value", i64},
+                                                           {"out_precision", i32},
+                                                           {"out_scale", i32},
+                                                       }));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("castDECIMAL_float64", i128,
+                                                       {
+                                                           {"value", f64},
+                                                           {"out_precision", i32},
+                                                           {"out_scale", i32},
+                                                       }));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("castDECIMAL_decimal128", i128,
+                                                       {
+                                                           {"x_value", i128},
+                                                           {"x_precision", i32},
+                                                           {"x_scale", i32},
+                                                           {"out_precision", i32},
+                                                           {"out_scale", i32},
+                                                       }));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("castBIGINT_decimal128", i64,
+                                                       {
+                                                           {"x_value", i128},
+                                                           {"x_precision", i32},
+                                                           {"x_scale", i32},
+                                                       }));
+
+  ARROW_RETURN_NOT_OK(decimal_ir->BuildDecimalFunction("castFLOAT8_decimal128", f64,
+                                                       {
+                                                           {"x_value", i128},
+                                                           {"x_precision", i32},
+                                                           {"x_scale", i32},
+                                                       }));
   return Status::OK();
 }
 
diff --git a/cpp/src/gandiva/decimal_ir.h b/cpp/src/gandiva/decimal_ir.h
index b1bf38d..c2a22b3 100644
--- a/cpp/src/gandiva/decimal_ir.h
+++ b/cpp/src/gandiva/decimal_ir.h
@@ -156,6 +156,9 @@ class DecimalIR : public FunctionIRBuilder {
   Status BuildCompare(const std::string& function_name,
                       llvm::ICmpInst::Predicate cmp_instruction);
 
+  Status BuildDecimalFunction(const std::string& function_name, llvm::Type* return_type,
+                              std::vector<NamedArg> in_types);
+
   // Add a trace in IR code.
   void AddTrace(const std::string& fmt, std::vector<llvm::Value*> args);
 
diff --git a/cpp/src/gandiva/function_registry_arithmetic.cc b/cpp/src/gandiva/function_registry_arithmetic.cc
index 04e9113..b6c5819 100644
--- a/cpp/src/gandiva/function_registry_arithmetic.cc
+++ b/cpp/src/gandiva/function_registry_arithmetic.cc
@@ -36,39 +36,42 @@ std::vector<NativeFunction> GetArithmeticFunctionRegistry() {
   static std::vector<NativeFunction> arithmetic_fn_registry_ = {
       UNARY_SAFE_NULL_IF_NULL(not, boolean, boolean),
       UNARY_SAFE_NULL_IF_NULL(castBIGINT, int32, int64),
+      UNARY_SAFE_NULL_IF_NULL(castBIGINT, decimal128, int64),
 
-      UNARY_CAST_TO_FLOAT32(int32),
-      UNARY_CAST_TO_FLOAT32(int64),
+      // cast to float32
+      UNARY_CAST_TO_FLOAT32(int32), UNARY_CAST_TO_FLOAT32(int64),
 
-      UNARY_CAST_TO_FLOAT64(int32),
-      UNARY_CAST_TO_FLOAT64(int64),
-      UNARY_CAST_TO_FLOAT64(float32),
+      // cast to float64
+      UNARY_CAST_TO_FLOAT64(int32), UNARY_CAST_TO_FLOAT64(int64),
+      UNARY_CAST_TO_FLOAT64(float32), UNARY_CAST_TO_FLOAT64(decimal128),
+
+      // cast to decimal
+      UNARY_SAFE_NULL_IF_NULL(castDECIMAL, int64, decimal128),
+      UNARY_SAFE_NULL_IF_NULL(castDECIMAL, float64, decimal128),
+      UNARY_SAFE_NULL_IF_NULL(castDECIMAL, decimal128, decimal128),
 
       UNARY_SAFE_NULL_IF_NULL(castDATE, int64, date64),
 
-      BINARY_SYMMETRIC_FN(add),
-      BINARY_SYMMETRIC_FN(subtract),
+      // add/sub/multiply/divide/mod
+      BINARY_SYMMETRIC_FN(add), BINARY_SYMMETRIC_FN(subtract),
       BINARY_SYMMETRIC_FN(multiply),
-
       NUMERIC_TYPES(BINARY_SYMMETRIC_UNSAFE_NULL_IF_NULL, divide),
       BINARY_GENERIC_SAFE_NULL_IF_NULL(mod, int64, int32, int32),
       BINARY_GENERIC_SAFE_NULL_IF_NULL(mod, int64, int64, int64),
-
       BINARY_SYMMETRIC_SAFE_NULL_IF_NULL(add, decimal128),
       BINARY_SYMMETRIC_SAFE_NULL_IF_NULL(subtract, decimal128),
       BINARY_SYMMETRIC_SAFE_NULL_IF_NULL(multiply, decimal128),
       BINARY_SYMMETRIC_UNSAFE_NULL_IF_NULL(divide, decimal128),
       BINARY_SYMMETRIC_UNSAFE_NULL_IF_NULL(mod, decimal128),
+
+      // compare functions
       BINARY_RELATIONAL_SAFE_NULL_IF_NULL(equal, decimal128),
       BINARY_RELATIONAL_SAFE_NULL_IF_NULL(not_equal, decimal128),
       BINARY_RELATIONAL_SAFE_NULL_IF_NULL(less_than, decimal128),
       BINARY_RELATIONAL_SAFE_NULL_IF_NULL(less_than_or_equal_to, decimal128),
       BINARY_RELATIONAL_SAFE_NULL_IF_NULL(greater_than, decimal128),
       BINARY_RELATIONAL_SAFE_NULL_IF_NULL(greater_than_or_equal_to, decimal128),
-
-      BINARY_RELATIONAL_BOOL_FN(equal),
-      BINARY_RELATIONAL_BOOL_FN(not_equal),
-
+      BINARY_RELATIONAL_BOOL_FN(equal), BINARY_RELATIONAL_BOOL_FN(not_equal),
       BINARY_RELATIONAL_BOOL_DATE_FN(less_than),
       BINARY_RELATIONAL_BOOL_DATE_FN(less_than_or_equal_to),
       BINARY_RELATIONAL_BOOL_DATE_FN(greater_than),
diff --git a/cpp/src/gandiva/function_registry_math_ops.cc b/cpp/src/gandiva/function_registry_math_ops.cc
index 31b4b13..2084b7b 100644
--- a/cpp/src/gandiva/function_registry_math_ops.cc
+++ b/cpp/src/gandiva/function_registry_math_ops.cc
@@ -59,7 +59,17 @@ std::vector<NativeFunction> GetMathOpsFunctionRegistry() {
       NUMERIC_TYPES(UNARY_SAFE_NULL_NEVER_BOOL, isnumeric),
 
       BINARY_SAFE_NULL_NEVER_BOOL_FN(is_distinct_from),
-      BINARY_SAFE_NULL_NEVER_BOOL_FN(is_not_distinct_from)};
+      BINARY_SAFE_NULL_NEVER_BOOL_FN(is_not_distinct_from),
+
+      // decimal functions
+      UNARY_SAFE_NULL_IF_NULL(abs, decimal128, decimal128),
+      UNARY_SAFE_NULL_IF_NULL(ceil, decimal128, decimal128),
+      UNARY_SAFE_NULL_IF_NULL(floor, decimal128, decimal128),
+      UNARY_SAFE_NULL_IF_NULL(round, decimal128, decimal128),
+      UNARY_SAFE_NULL_IF_NULL(truncate, decimal128, decimal128),
+      BINARY_GENERIC_SAFE_NULL_IF_NULL(round, decimal128, int32, decimal128),
+      BINARY_GENERIC_SAFE_NULL_IF_NULL(truncate, decimal128, int32, decimal128),
+  };
 
   return math_fn_registry_;
 }
diff --git a/cpp/src/gandiva/precompiled/decimal_ops.cc b/cpp/src/gandiva/precompiled/decimal_ops.cc
index 9bf643f..6f20f59 100644
--- a/cpp/src/gandiva/precompiled/decimal_ops.cc
+++ b/cpp/src/gandiva/precompiled/decimal_ops.cc
@@ -20,6 +20,8 @@
 #include "gandiva/precompiled/decimal_ops.h"
 
 #include <algorithm>
+#include <cmath>
+#include <limits>
 
 #include "gandiva/decimal_type_util.h"
 #include "gandiva/decimal_xlarge.h"
@@ -463,5 +465,242 @@ int32_t Compare(const BasicDecimalScalar128& x, const BasicDecimalScalar128& y)
   }
 }
 
+#define DECIMAL_OVERFLOW_IF(condition, overflow) \
+  do {                                           \
+    if (condition) {                             \
+      *overflow = true;                          \
+      return 0;                                  \
+    }                                            \
+  } while (0)
+
+static BasicDecimal128 GetMaxValue(int32_t precision) {
+  return BasicDecimal128::GetScaleMultiplier(precision) - 1;
+}
+
+// Compute the double scale multipliers once.
+static std::array<double, DecimalTypeUtil::kMaxPrecision + 1> kDoubleScaleMultipliers =
+    ([]() -> std::array<double, DecimalTypeUtil::kMaxPrecision + 1> {
+      std::array<double, DecimalTypeUtil::kMaxPrecision + 1> values;
+      values[0] = 1.0;
+      for (int32_t idx = 1; idx <= DecimalTypeUtil::kMaxPrecision; idx++) {
+        values[idx] = values[idx - 1] * 10;
+      }
+      return values;
+    })();
+
+BasicDecimal128 FromDouble(double in, int32_t precision, int32_t scale, bool* overflow) {
+  // Multiply decimal with the scale
+  auto unscaled = in * kDoubleScaleMultipliers[scale];
+  unscaled = std::round(unscaled);
+  DECIMAL_OVERFLOW_IF(std::isnan(unscaled) || std::fabs(unscaled) < std::fabs(in),
+                      overflow);
+
+  // convert scaled double to int128
+  int32_t sign = unscaled < 0 ? -1 : 1;
+  auto unscaled_abs = std::abs(unscaled);
+
+  // overflow if > 2^127 - 1
+  DECIMAL_OVERFLOW_IF(unscaled_abs > std::ldexp(static_cast<double>(1), 127) - 1,
+                      overflow);
+
+  uint64_t high_bits = static_cast<uint64_t>(std::ldexp(unscaled_abs, -64));
+  uint64_t low_bits = static_cast<uint64_t>(
+      unscaled_abs - std::ldexp(static_cast<double>(high_bits), 64));
+
+  auto result = BasicDecimal128(static_cast<int64_t>(high_bits), low_bits);
+
+  // overflow if > max value based on precision
+  DECIMAL_OVERFLOW_IF(result > GetMaxValue(precision), overflow);
+  return result * sign;
+}
+
+double ToDouble(const BasicDecimalScalar128& in, bool* overflow) {
+  // convert int128 to double
+  int64_t sign = in.value().Sign();
+  auto value_abs = BasicDecimal128::Abs(in.value());
+  double unscaled = static_cast<double>(value_abs.low_bits()) +
+                    std::ldexp(static_cast<double>(value_abs.high_bits()), 64);
+
+  // scale double.
+  return (unscaled * sign) / kDoubleScaleMultipliers[in.scale()];
+}
+
+BasicDecimal128 FromInt64(int64_t in, int32_t precision, int32_t scale, bool* overflow) {
+  // check if multiplying by scale will cause an overflow.
+  DECIMAL_OVERFLOW_IF(std::abs(in) > GetMaxValue(precision - scale), overflow);
+  return in * BasicDecimal128::GetScaleMultiplier(scale);
+}
+
+int64_t ToInt64(const BasicDecimalScalar128& in, bool* overflow) {
+  BasicDecimal128 whole, fraction;
+
+  in.value().GetWholeAndFraction(in.scale(), &whole, &fraction);
+  DECIMAL_OVERFLOW_IF((whole > std::numeric_limits<int64_t>::max()) ||
+                          (whole < std::numeric_limits<int64_t>::min()),
+                      overflow);
+  return static_cast<int64_t>(whole.low_bits());
+}
+
+// Helper function to modify the scale and/or precision of a decimal value.
+static BasicDecimal128 ModifyScaleAndPrecision(const BasicDecimalScalar128& x,
+                                               int32_t out_precision, int32_t out_scale,
+                                               bool* overflow) {
+  int32_t delta_scale = out_scale - x.scale();
+  if (delta_scale >= 0) {
+    // check if multiplying by delta_scale will cause an overflow.
+    DECIMAL_OVERFLOW_IF(
+        BasicDecimal128::Abs(x.value()) > GetMaxValue(out_precision - delta_scale),
+        overflow);
+    return x.value().IncreaseScaleBy(delta_scale);
+  } else {
+    // Do not do any rounding, that is handled by the caller.
+    return x.value().ReduceScaleBy(-delta_scale, false);
+  }
+}
+
+enum RoundType {
+  kRoundTypeCeil,         // +1 if +ve and trailing value is > 0, else no rounding.
+  kRoundTypeFloor,        // -1 if -ve and trailing value is < 0, else no rounding.
+  kRoundTypeTrunc,        // no rounding, truncate the trailing digits.
+  kRoundTypeHalfRoundUp,  // if +ve and trailing value is >= half of base, +1.
+                          // else if -ve and trailing value is >= half of base, -1.
+};
+
+// Compute the rounding delta for the givven rounding type.
+static int32_t ComputeRoundingDelta(const BasicDecimal128& x, int32_t x_scale,
+                                    int32_t out_scale, RoundType type) {
+  if (type == kRoundTypeTrunc ||  // no rounding for this type.
+      out_scale >= x_scale) {     // no digits dropped, so no rounding.
+    return 0;
+  }
+
+  int32_t result = 0;
+  switch (type) {
+    case kRoundTypeHalfRoundUp: {
+      auto base = BasicDecimal128::GetScaleMultiplier(x_scale - out_scale);
+      auto trailing = x % base;
+      if (trailing == 0) {
+        result = 0;
+      } else if (trailing.Abs() < base / 2) {
+        result = 0;
+      } else {
+        result = (x < 0) ? -1 : 1;
+      }
+      break;
+    }
+
+    case kRoundTypeCeil:
+      if (x < 0) {
+        // no rounding for -ve
+        result = 0;
+      } else {
+        auto base = BasicDecimal128::GetScaleMultiplier(x_scale - out_scale);
+        auto trailing = x % base;
+        result = (trailing == 0) ? 0 : 1;
+      }
+      break;
+
+    case kRoundTypeFloor:
+      if (x > 0) {
+        // no rounding for +ve
+        result = 0;
+      } else {
+        auto base = BasicDecimal128::GetScaleMultiplier(x_scale - out_scale);
+        auto trailing = x % base;
+        result = (trailing == 0) ? 0 : -1;
+      }
+      break;
+
+    case kRoundTypeTrunc:
+      break;
+  }
+  return result;
+}
+
+// Modify the scale and round.
+static BasicDecimal128 RoundWithPositiveScale(const BasicDecimalScalar128& x,
+                                              int32_t out_precision, int32_t out_scale,
+                                              RoundType round_type, bool* overflow) {
+  DCHECK_GE(out_scale, 0);
+
+  auto scaled = ModifyScaleAndPrecision(x, out_precision, out_scale, overflow);
+  auto delta = ComputeRoundingDelta(x.value(), x.scale(), out_scale, round_type);
+  if (delta == 0) {
+    return scaled;
+  }
+
+  // If there is a rounding delta, the output scale must be less than the input scale.
+  // That means atleast one digit is dropped after the decimal. The delta add can add
+  // utmost one digit before the decimal. So, overflow will occur only if the output
+  // precision has changed.
+  DCHECK_GT(x.scale(), out_scale);
+  auto result = scaled + delta;
+  DECIMAL_OVERFLOW_IF(out_precision < x.precision() &&
+                          BasicDecimal128::Abs(result) > GetMaxValue(out_precision),
+                      overflow);
+  return result;
+}
+
+// Modify scale to drop all digits to the right of the decimal and round.
+// Then, zero out 'rounding_scale' number of digits to the left of the decimal point.
+static BasicDecimal128 RoundWithNegativeScale(const BasicDecimalScalar128& x,
+                                              int32_t out_precision,
+                                              int32_t rounding_scale,
+                                              RoundType round_type, bool* overflow) {
+  DCHECK_LT(rounding_scale, 0);
+
+  // get rid of the fractional part.
+  auto scaled = ModifyScaleAndPrecision(x, out_precision, 0, overflow);
+  auto rounding_delta = ComputeRoundingDelta(scaled, 0, -rounding_scale, round_type);
+
+  auto base = BasicDecimal128::GetScaleMultiplier(-rounding_scale);
+  auto delta = rounding_delta * base - (scaled % base);
+  DECIMAL_OVERFLOW_IF(BasicDecimal128::Abs(scaled) >
+                          GetMaxValue(out_precision) - BasicDecimal128::Abs(delta),
+                      overflow);
+  return scaled + delta;
+}
+
+BasicDecimal128 Round(const BasicDecimalScalar128& x, int32_t out_scale, bool* overflow) {
+  if (out_scale < 0) {
+    return RoundWithNegativeScale(x, x.precision(), out_scale,
+                                  RoundType::kRoundTypeHalfRoundUp, overflow);
+  } else {
+    return RoundWithPositiveScale(x, x.precision(), out_scale,
+                                  RoundType::kRoundTypeHalfRoundUp, overflow);
+  }
+}
+
+BasicDecimal128 Truncate(const BasicDecimalScalar128& x, int32_t out_scale,
+                         bool* overflow) {
+  if (out_scale < 0) {
+    return RoundWithNegativeScale(x, x.precision(), out_scale, RoundType::kRoundTypeTrunc,
+                                  overflow);
+  } else {
+    return RoundWithPositiveScale(x, x.precision(), out_scale, RoundType::kRoundTypeTrunc,
+                                  overflow);
+  }
+}
+
+BasicDecimal128 Ceil(const BasicDecimalScalar128& x, bool* overflow) {
+  return RoundWithPositiveScale(x, x.precision(), 0, RoundType::kRoundTypeCeil, overflow);
+}
+
+BasicDecimal128 Floor(const BasicDecimalScalar128& x, bool* overflow) {
+  return RoundWithPositiveScale(x, x.precision(), 0, RoundType::kRoundTypeFloor,
+                                overflow);
+}
+
+BasicDecimal128 Convert(const BasicDecimalScalar128& x, int32_t out_precision,
+                        int32_t out_scale, bool* overflow) {
+  DCHECK_GE(out_scale, 0);
+  DCHECK_LE(out_scale, DecimalTypeUtil::kMaxScale);
+  DCHECK_GT(out_precision, 0);
+  DCHECK_LE(out_precision, DecimalTypeUtil::kMaxScale);
+
+  return RoundWithPositiveScale(x, out_precision, out_scale,
+                                RoundType::kRoundTypeHalfRoundUp, overflow);
+}
+
 }  // namespace decimalops
 }  // namespace gandiva
diff --git a/cpp/src/gandiva/precompiled/decimal_ops.h b/cpp/src/gandiva/precompiled/decimal_ops.h
index 19417a2..1c1c9de 100644
--- a/cpp/src/gandiva/precompiled/decimal_ops.h
+++ b/cpp/src/gandiva/precompiled/decimal_ops.h
@@ -56,5 +56,34 @@ arrow::BasicDecimal128 Mod(int64_t context, const BasicDecimalScalar128& x,
 /// -1 if x < y
 int32_t Compare(const BasicDecimalScalar128& x, const BasicDecimalScalar128& y);
 
+/// Convert to decimal from double.
+BasicDecimal128 FromDouble(double in, int32_t precision, int32_t scale, bool* overflow);
+
+/// Convert from decimal to double.
+double ToDouble(const BasicDecimalScalar128& in, bool* overflow);
+
+/// Convert to decimal from int64.
+BasicDecimal128 FromInt64(int64_t in, int32_t precision, int32_t scale, bool* overflow);
+
+/// Convert from decimal to int64
+int64_t ToInt64(const BasicDecimalScalar128& in, bool* overflow);
+
+/// Convert from one decimal scale/precision to another.
+BasicDecimal128 Convert(const BasicDecimalScalar128& x, int32_t out_precision,
+                        int32_t out_scale, bool* overflow);
+
+/// round decimal.
+BasicDecimal128 Round(const BasicDecimalScalar128& x, int32_t out_scale, bool* overflow);
+
+/// truncate decimal.
+BasicDecimal128 Truncate(const BasicDecimalScalar128& x, int32_t out_scale,
+                         bool* overflow);
+
+/// ceil decimal
+BasicDecimal128 Ceil(const BasicDecimalScalar128& x, bool* overflow);
+
+/// floor decimal
+BasicDecimal128 Floor(const BasicDecimalScalar128& x, bool* overflow);
+
 }  // namespace decimalops
 }  // namespace gandiva
diff --git a/cpp/src/gandiva/precompiled/decimal_ops_test.cc b/cpp/src/gandiva/precompiled/decimal_ops_test.cc
index f3b22ff..6c8d762 100644
--- a/cpp/src/gandiva/precompiled/decimal_ops_test.cc
+++ b/cpp/src/gandiva/precompiled/decimal_ops_test.cc
@@ -17,7 +17,10 @@
 
 #include <gtest/gtest.h>
 #include <algorithm>
+#include <limits>
 #include <memory>
+#include <tuple>
+#include <vector>
 
 #include "arrow/testing/gtest_util.h"
 #include "gandiva/decimal_scalar.h"
@@ -547,4 +550,503 @@ TEST_F(TestDecimalSql, Compare) {
                                     DecimalScalar128{kThirtyEight9s, 38, 4}));
 }
 
+TEST_F(TestDecimalSql, Round) {
+  // expected, input, rounding_scale, overflow
+  using TupleType = std::tuple<BasicDecimal128, DecimalScalar128, int32_t, bool>;
+  std::vector<TupleType> test_values = {
+      // examples from
+      // https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_round
+      std::make_tuple(BasicDecimal128{-1}, DecimalScalar128{-123, 38, 2}, 0, false),
+      std::make_tuple(BasicDecimal128{-2}, DecimalScalar128{-158, 38, 2}, 0, false),
+      std::make_tuple(BasicDecimal128{2}, DecimalScalar128{158, 38, 2}, 0, false),
+      std::make_tuple(BasicDecimal128{-13}, DecimalScalar128{-1298, 38, 3}, 1, false),
+      std::make_tuple(BasicDecimal128{-1}, DecimalScalar128{-1298, 38, 3}, 0, false),
+      std::make_tuple(BasicDecimal128{20}, DecimalScalar128{23298, 38, 3}, -1, false),
+      std::make_tuple(BasicDecimal128{3}, DecimalScalar128{25, 38, 1}, 0, false),
+
+      // border cases
+      std::make_tuple(BasicDecimal128{INT64_MIN / 100},
+                      DecimalScalar128{INT64_MIN, 38, 2}, 0, false),
+
+      std::make_tuple(INT64_MIN, DecimalScalar128{INT64_MIN, 38, 0}, 0, false),
+      std::make_tuple(BasicDecimal128{0, 0}, DecimalScalar128{0, 0, 38, 2}, 0, false),
+      std::make_tuple(INT64_MAX, DecimalScalar128{INT64_MAX, 38, 0}, 0, false),
+
+      std::make_tuple(BasicDecimal128(INT64_MAX / 100),
+                      DecimalScalar128{INT64_MAX, 38, 2}, 0, false),
+
+      // large scales
+      std::make_tuple(BasicDecimal128{0, 0}, DecimalScalar128{12345, 38, 16}, 0, false),
+      std::make_tuple(
+          BasicDecimal128{124},
+          DecimalScalar128{BasicDecimal128{12389}.IncreaseScaleBy(14), 38, 16}, 0, false),
+      std::make_tuple(
+          BasicDecimal128{-124},
+          DecimalScalar128{BasicDecimal128{-12389}.IncreaseScaleBy(14), 38, 16}, 0,
+          false),
+      std::make_tuple(
+          BasicDecimal128{124},
+          DecimalScalar128{BasicDecimal128{12389}.IncreaseScaleBy(30), 38, 32}, 0, false),
+      std::make_tuple(
+          BasicDecimal128{-124},
+          DecimalScalar128{BasicDecimal128{-12389}.IncreaseScaleBy(30), 38, 32}, 0,
+          false),
+
+      // overflow
+      std::make_tuple(
+          BasicDecimal128{0, 0},
+          DecimalScalar128{BasicDecimal128{12389}.IncreaseScaleBy(32), 38, 32}, 35, true),
+      std::make_tuple(
+          BasicDecimal128{0, 0},
+          DecimalScalar128{BasicDecimal128{-12389}.IncreaseScaleBy(32), 38, 32}, 35,
+          true),
+  };
+
+  for (auto iter : test_values) {
+    auto expected = std::get<0>(iter);
+    auto input = std::get<1>(iter);
+    auto rounding_scale = std::get<2>(iter);
+    auto expected_overflow = std::get<3>(iter);
+    bool overflow = false;
+
+    EXPECT_EQ(expected, decimalops::Round(input, rounding_scale, &overflow))
+        << "  failed on input " << input << "  rounding scale " << rounding_scale;
+    if (expected_overflow) {
+      ASSERT_TRUE(overflow) << "overflow expected for input " << input;
+    } else {
+      ASSERT_FALSE(overflow) << "overflow not expected for input " << input;
+    }
+  }
+}
+
+TEST_F(TestDecimalSql, Truncate) {
+  // expected, input, rounding_scale, overflow
+  using TupleType = std::tuple<BasicDecimal128, DecimalScalar128, int32_t, bool>;
+  std::vector<TupleType> test_values = {
+      // examples from
+      // https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_truncate
+      std::make_tuple(BasicDecimal128{12}, DecimalScalar128{1223, 38, 3}, 1, false),
+      std::make_tuple(BasicDecimal128{19}, DecimalScalar128{1999, 38, 3}, 1, false),
+      std::make_tuple(BasicDecimal128{1}, DecimalScalar128{1999, 38, 3}, 0, false),
+      std::make_tuple(BasicDecimal128{-19}, DecimalScalar128{-1999, 38, 3}, 1, false),
+      std::make_tuple(BasicDecimal128{100}, DecimalScalar128{122, 38, 0}, -2, false),
+      std::make_tuple(BasicDecimal128{1028}, DecimalScalar128{1028, 38, 0}, 0, false),
+
+      // border cases
+      std::make_tuple(BasicDecimal128{INT64_MIN / 100},
+                      DecimalScalar128{INT64_MIN, 38, 2}, 0, false),
+
+      std::make_tuple(INT64_MIN, DecimalScalar128{INT64_MIN, 38, 0}, 0, false),
+      std::make_tuple(BasicDecimal128{0, 0}, DecimalScalar128{0, 0, 38, 2}, 0, false),
+      std::make_tuple(INT64_MAX, DecimalScalar128{INT64_MAX, 38, 0}, 0, false),
+
+      std::make_tuple(BasicDecimal128(INT64_MAX / 100),
+                      DecimalScalar128{INT64_MAX, 38, 2}, 0, false),
+
+      // large scales
+      std::make_tuple(BasicDecimal128{0, 0}, DecimalScalar128{12345, 38, 16}, 0, false),
+      std::make_tuple(
+          BasicDecimal128{123},
+          DecimalScalar128{BasicDecimal128{12389}.IncreaseScaleBy(14), 38, 16}, 0, false),
+      std::make_tuple(
+          BasicDecimal128{-123},
+          DecimalScalar128{BasicDecimal128{-12389}.IncreaseScaleBy(14), 38, 16}, 0,
+          false),
+      std::make_tuple(
+          BasicDecimal128{123},
+          DecimalScalar128{BasicDecimal128{12389}.IncreaseScaleBy(30), 38, 32}, 0, false),
+      std::make_tuple(
+          BasicDecimal128{-123},
+          DecimalScalar128{BasicDecimal128{-12389}.IncreaseScaleBy(30), 38, 32}, 0,
+          false),
+
+      // overflow
+      std::make_tuple(
+          BasicDecimal128{0, 0},
+          DecimalScalar128{BasicDecimal128{12389}.IncreaseScaleBy(32), 38, 32}, 35, true),
+      std::make_tuple(
+          BasicDecimal128{0, 0},
+          DecimalScalar128{BasicDecimal128{-12389}.IncreaseScaleBy(32), 38, 32}, 35,
+          true),
+  };
+
+  for (auto iter : test_values) {
+    auto expected = std::get<0>(iter);
+    auto input = std::get<1>(iter);
+    auto rounding_scale = std::get<2>(iter);
+    auto expected_overflow = std::get<3>(iter);
+    bool overflow = false;
+
+    EXPECT_EQ(expected, decimalops::Truncate(input, rounding_scale, &overflow))
+        << "  failed on input " << input << "  rounding scale " << rounding_scale;
+    if (expected_overflow) {
+      ASSERT_TRUE(overflow) << "overflow expected for input " << input;
+    } else {
+      ASSERT_FALSE(overflow) << "overflow not expected for input " << input;
+    }
+  }
+}
+
+TEST_F(TestDecimalSql, Ceil) {
+  // expected, input, overflow
+  std::vector<std::tuple<BasicDecimal128, DecimalScalar128, bool>> test_values = {
+      // https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_ceil
+      std::make_tuple(2, DecimalScalar128{123, 38, 2}, false),
+      std::make_tuple(-1, DecimalScalar128{-123, 38, 2}, false),
+
+      // border cases
+      std::make_tuple(BasicDecimal128{INT64_MIN / 100},
+                      DecimalScalar128{INT64_MIN, 38, 2}, false),
+
+      std::make_tuple(INT64_MIN, DecimalScalar128{INT64_MIN, 38, 0}, false),
+      std::make_tuple(BasicDecimal128{0, 0}, DecimalScalar128{0, 0, 38, 2}, false),
+      std::make_tuple(INT64_MAX, DecimalScalar128{INT64_MAX, 38, 0}, false),
+
+      std::make_tuple(BasicDecimal128(INT64_MAX / 100 + 1),
+                      DecimalScalar128{INT64_MAX, 38, 2}, false),
+
+      // large scales
+      std::make_tuple(BasicDecimal128{0, 1}, DecimalScalar128{12345, 38, 16}, false),
+      std::make_tuple(
+          BasicDecimal128{124},
+          DecimalScalar128{BasicDecimal128{12389}.IncreaseScaleBy(14), 38, 16}, false),
+      std::make_tuple(
+          BasicDecimal128{-123},
+          DecimalScalar128{BasicDecimal128{-12389}.IncreaseScaleBy(14), 38, 16}, false),
+      std::make_tuple(
+          BasicDecimal128{124},
+          DecimalScalar128{BasicDecimal128{12389}.IncreaseScaleBy(30), 38, 32}, false),
+      std::make_tuple(
+          BasicDecimal128{-123},
+          DecimalScalar128{BasicDecimal128{-12389}.IncreaseScaleBy(30), 38, 32}, false),
+  };
+
+  for (auto iter : test_values) {
+    auto expected = std::get<0>(iter);
+    auto input = std::get<1>(iter);
+    auto expected_overflow = std::get<2>(iter);
+    bool overflow = false;
+
+    EXPECT_EQ(expected, decimalops::Ceil(input, &overflow))
+        << "  failed on input " << input;
+    if (expected_overflow) {
+      ASSERT_TRUE(overflow) << "overflow expected for input " << input;
+    } else {
+      ASSERT_FALSE(overflow) << "overflow not expected for input " << input;
+    }
+  }
+}
+
+TEST_F(TestDecimalSql, Floor) {
+  // expected, input, overflow
+  std::vector<std::tuple<BasicDecimal128, DecimalScalar128, bool>> test_values = {
+      // https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_floor
+      std::make_tuple(1, DecimalScalar128{123, 38, 2}, false),
+      std::make_tuple(-2, DecimalScalar128{-123, 38, 2}, false),
+
+      // border cases
+      std::make_tuple(BasicDecimal128{INT64_MIN / 100 - 1},
+                      DecimalScalar128{INT64_MIN, 38, 2}, false),
+
+      std::make_tuple(INT64_MIN, DecimalScalar128{INT64_MIN, 38, 0}, false),
+      std::make_tuple(BasicDecimal128{0, 0}, DecimalScalar128{0, 0, 38, 2}, false),
+      std::make_tuple(INT64_MAX, DecimalScalar128{INT64_MAX, 38, 0}, false),
+
+      std::make_tuple(BasicDecimal128{INT64_MAX / 100},
+                      DecimalScalar128{INT64_MAX, 38, 2}, false),
+
+      // large scales
+      std::make_tuple(BasicDecimal128{0, 0}, DecimalScalar128{12345, 38, 16}, false),
+      std::make_tuple(
+          BasicDecimal128{123},
+          DecimalScalar128{BasicDecimal128{12389}.IncreaseScaleBy(14), 38, 16}, false),
+      std::make_tuple(
+          BasicDecimal128{-124},
+          DecimalScalar128{BasicDecimal128{-12389}.IncreaseScaleBy(14), 38, 16}, false),
+      std::make_tuple(
+          BasicDecimal128{123},
+          DecimalScalar128{BasicDecimal128{12389}.IncreaseScaleBy(30), 38, 32}, false),
+      std::make_tuple(
+          BasicDecimal128{-124},
+          DecimalScalar128{BasicDecimal128{-12389}.IncreaseScaleBy(30), 38, 32}, false),
+  };
+
+  for (auto iter : test_values) {
+    auto expected = std::get<0>(iter);
+    auto input = std::get<1>(iter);
+    auto expected_overflow = std::get<2>(iter);
+    bool overflow = false;
+
+    EXPECT_EQ(expected, decimalops::Floor(input, &overflow))
+        << "  failed on input " << input;
+    if (expected_overflow) {
+      ASSERT_TRUE(overflow) << "overflow expected for input " << input;
+    } else {
+      ASSERT_FALSE(overflow) << "overflow not expected for input " << input;
+    }
+  }
+}
+
+TEST_F(TestDecimalSql, Convert) {
+  // expected, input, overflow
+  std::vector<std::tuple<DecimalScalar128, DecimalScalar128, bool>> test_values = {
+      // simple cases
+      std::make_tuple(DecimalScalar128{12, 38, 1}, DecimalScalar128{123, 38, 2}, false),
+      std::make_tuple(DecimalScalar128{1230, 38, 3}, DecimalScalar128{123, 38, 2}, false),
+      std::make_tuple(DecimalScalar128{123, 38, 2}, DecimalScalar128{123, 38, 2}, false),
+
+      std::make_tuple(DecimalScalar128{-12, 38, 1}, DecimalScalar128{-123, 38, 2}, false),
+      std::make_tuple(DecimalScalar128{-1230, 38, 3}, DecimalScalar128{-123, 38, 2},
+                      false),
+      std::make_tuple(DecimalScalar128{-123, 38, 2}, DecimalScalar128{-123, 38, 2},
+                      false),
+
+      // border cases
+      std::make_tuple(
+          DecimalScalar128{BasicDecimal128(INT64_MIN).ReduceScaleBy(1), 38, 1},
+          DecimalScalar128{INT64_MIN, 38, 2}, false),
+      std::make_tuple(
+          DecimalScalar128{BasicDecimal128(INT64_MIN).IncreaseScaleBy(1), 38, 3},
+          DecimalScalar128{INT64_MIN, 38, 2}, false),
+      std::make_tuple(DecimalScalar128{-3, 38, 1}, DecimalScalar128{-32, 38, 2}, false),
+      std::make_tuple(DecimalScalar128{0, 0, 38, 1}, DecimalScalar128{0, 0, 38, 2},
+                      false),
+      std::make_tuple(DecimalScalar128{3, 38, 1}, DecimalScalar128{32, 38, 2}, false),
+      std::make_tuple(
+          DecimalScalar128{BasicDecimal128(INT64_MAX).ReduceScaleBy(1), 38, 1},
+          DecimalScalar128{INT64_MAX, 38, 2}, false),
+      std::make_tuple(
+          DecimalScalar128{BasicDecimal128(INT64_MAX).IncreaseScaleBy(1), 38, 3},
+          DecimalScalar128{INT64_MAX, 38, 2}, false),
+
+      // large scales
+      std::make_tuple(DecimalScalar128{BasicDecimal128(123).IncreaseScaleBy(16), 38, 18},
+                      DecimalScalar128{123, 38, 2}, false),
+      std::make_tuple(DecimalScalar128{BasicDecimal128(-123).IncreaseScaleBy(16), 38, 18},
+                      DecimalScalar128{-123, 38, 2}, false),
+      std::make_tuple(DecimalScalar128{BasicDecimal128(123).IncreaseScaleBy(30), 38, 32},
+                      DecimalScalar128{123, 38, 2}, false),
+      std::make_tuple(DecimalScalar128{BasicDecimal128(-123).IncreaseScaleBy(30), 38, 32},
+                      DecimalScalar128{-123, 38, 2}, false),
+
+      // overflow due to scaling up.
+      std::make_tuple(DecimalScalar128{0, 0, 38, 36}, DecimalScalar128{12345, 38, 2},
+                      true),
+      std::make_tuple(DecimalScalar128{0, 0, 38, 36}, DecimalScalar128{-12345, 38, 2},
+                      true),
+
+      // overflow due to precision.
+      std::make_tuple(DecimalScalar128{0, 0, 5, 3}, DecimalScalar128{12345, 5, 2}, true),
+  };
+
+  for (auto iter : test_values) {
+    auto expected = std::get<0>(iter);
+    auto input = std::get<1>(iter);
+    auto expected_overflow = std::get<2>(iter);
+    bool overflow = false;
+
+    EXPECT_EQ(expected.value(), decimalops::Convert(input, expected.precision(),
+                                                    expected.scale(), &overflow))
+        << "  failed on input " << input;
+
+    if (expected_overflow) {
+      ASSERT_TRUE(overflow) << "overflow expected for input " << input;
+    } else {
+      ASSERT_FALSE(overflow) << "overflow not expected for input " << input;
+    }
+  }
+}
+
+// double can store upto this integer value without losing precision
+static const int64_t kMaxDoubleInt = 1ull << 53;
+
+TEST_F(TestDecimalSql, FromDouble) {
+  // expected, input, overflow
+  std::vector<std::tuple<DecimalScalar128, double, bool>> test_values = {
+      // simple cases
+      std::make_tuple(DecimalScalar128{-16285, 38, 3}, -16.285, false),
+      std::make_tuple(DecimalScalar128{-162850, 38, 4}, -16.285, false),
+      std::make_tuple(DecimalScalar128{-1629, 38, 2}, -16.285, false),
+
+      std::make_tuple(DecimalScalar128{16285, 38, 3}, 16.285, false),
+      std::make_tuple(DecimalScalar128{162850, 38, 4}, 16.285, false),
+      std::make_tuple(DecimalScalar128{1629, 38, 2}, 16.285, false),
+
+      // border cases
+      std::make_tuple(DecimalScalar128{-kMaxDoubleInt, 38, 0},
+                      static_cast<double>(-kMaxDoubleInt), false),
+      std::make_tuple(DecimalScalar128{-32, 38, 0}, -32, false),
+      std::make_tuple(DecimalScalar128{0, 0, 38, 0}, 0, false),
+      std::make_tuple(DecimalScalar128{32, 38, 0}, 32, false),
+      std::make_tuple(DecimalScalar128{kMaxDoubleInt, 38, 0},
+                      static_cast<double>(kMaxDoubleInt), false),
+
+      // large scales
+      std::make_tuple(DecimalScalar128{123, 38, 16}, 1.23E-14, false),
+      std::make_tuple(DecimalScalar128{123, 38, 32}, 1.23E-30, false),
+      std::make_tuple(DecimalScalar128{1230, 38, 33}, 1.23E-30, false),
+      std::make_tuple(DecimalScalar128{123, 38, 38}, 1.23E-36, false),
+
+      // overflow due to very low double
+      std::make_tuple(DecimalScalar128{0, 0, 38, 0}, std::numeric_limits<double>::min(),
+                      true),
+      // overflow due to very high double
+      std::make_tuple(DecimalScalar128{0, 0, 38, 0}, std::numeric_limits<double>::max(),
+                      true),
+      // overflow due to scaling up.
+      std::make_tuple(DecimalScalar128{0, 0, 38, 36}, 123.45, true),
+      // overflow due to precision.
+      std::make_tuple(DecimalScalar128{0, 0, 4, 2}, 12345.67, true),
+  };
+
+  for (auto iter : test_values) {
+    auto dscalar = std::get<0>(iter);
+    auto input = std::get<1>(iter);
+    auto expected_overflow = std::get<2>(iter);
+    bool overflow = false;
+
+    EXPECT_EQ(dscalar.value(), decimalops::FromDouble(input, dscalar.precision(),
+                                                      dscalar.scale(), &overflow))
+        << "  failed on input " << input;
+
+    if (expected_overflow) {
+      ASSERT_TRUE(overflow) << "overflow expected for input " << input;
+    } else {
+      ASSERT_FALSE(overflow) << "overflow not expected for input " << input;
+    }
+  }
+}
+
+#define EXPECT_FUZZY_EQ(x, y) \
+  EXPECT_TRUE(x - y <= 0.00001) << "expected " << x << ", got " << y
+
+TEST_F(TestDecimalSql, ToDouble) {
+  // expected, input, overflow
+  std::vector<std::tuple<double, DecimalScalar128>> test_values = {
+      // simple ones
+      std::make_tuple(-16.285, DecimalScalar128{-16285, 38, 3}),
+      std::make_tuple(-162.85, DecimalScalar128{-16285, 38, 2}),
+      std::make_tuple(-1.6285, DecimalScalar128{-16285, 38, 4}),
+
+      // large scales
+      std::make_tuple(1.23E-14, DecimalScalar128{123, 38, 16}),
+      std::make_tuple(1.23E-30, DecimalScalar128{123, 38, 32}),
+      std::make_tuple(1.23E-36, DecimalScalar128{123, 38, 38}),
+
+      // border cases
+      std::make_tuple(static_cast<double>(-kMaxDoubleInt),
+                      DecimalScalar128{-kMaxDoubleInt, 38, 0}),
+      std::make_tuple(-32, DecimalScalar128{-32, 38, 0}),
+      std::make_tuple(0, DecimalScalar128{0, 0, 38, 0}),
+      std::make_tuple(32, DecimalScalar128{32, 38, 0}),
+      std::make_tuple(static_cast<double>(kMaxDoubleInt),
+                      DecimalScalar128{kMaxDoubleInt, 38, 0}),
+  };
+  for (auto iter : test_values) {
+    auto input = std::get<1>(iter);
+    bool overflow = false;
+
+    EXPECT_FUZZY_EQ(std::get<0>(iter), decimalops::ToDouble(input, &overflow));
+    ASSERT_FALSE(overflow) << "overflow not expected for input " << input;
+  }
+}
+
+TEST_F(TestDecimalSql, FromInt64) {
+  // expected, input, overflow
+  std::vector<std::tuple<DecimalScalar128, int64_t, bool>> test_values = {
+      // simple cases
+      std::make_tuple(DecimalScalar128{-16000, 38, 3}, -16, false),
+      std::make_tuple(DecimalScalar128{-160000, 38, 4}, -16, false),
+      std::make_tuple(DecimalScalar128{-1600, 38, 2}, -16, false),
+
+      std::make_tuple(DecimalScalar128{16000, 38, 3}, 16, false),
+      std::make_tuple(DecimalScalar128{160000, 38, 4}, 16, false),
+      std::make_tuple(DecimalScalar128{1600, 38, 2}, 16, false),
+
+      // border cases
+      std::make_tuple(DecimalScalar128{INT64_MIN, 38, 0}, INT64_MIN, false),
+      std::make_tuple(DecimalScalar128{-32, 38, 0}, -32, false),
+      std::make_tuple(DecimalScalar128{0, 0, 38, 0}, 0, false),
+      std::make_tuple(DecimalScalar128{32, 38, 0}, 32, false),
+      std::make_tuple(DecimalScalar128{INT64_MAX, 38, 0}, INT64_MAX, false),
+
+      // large scales
+      std::make_tuple(DecimalScalar128{BasicDecimal128(123).IncreaseScaleBy(16), 38, 16},
+                      123, false),
+      std::make_tuple(DecimalScalar128{BasicDecimal128(123).IncreaseScaleBy(32), 38, 32},
+                      123, false),
+      std::make_tuple(DecimalScalar128{BasicDecimal128(-123).IncreaseScaleBy(16), 38, 16},
+                      -123, false),
+      std::make_tuple(DecimalScalar128{BasicDecimal128(-123).IncreaseScaleBy(32), 38, 32},
+                      -123, false),
+
+      // overflow due to scaling up.
+      std::make_tuple(DecimalScalar128{0, 0, 38, 36}, 123, true),
+      // overflow due to precision.
+      std::make_tuple(DecimalScalar128{0, 0, 4, 2}, 12345, true),
+  };
+
+  for (auto iter : test_values) {
+    auto dscalar = std::get<0>(iter);
+    auto input = std::get<1>(iter);
+    auto expected_overflow = std::get<2>(iter);
+    bool overflow = false;
+
+    EXPECT_EQ(dscalar.value(), decimalops::FromInt64(input, dscalar.precision(),
+                                                     dscalar.scale(), &overflow))
+        << "  failed on input " << input;
+
+    if (expected_overflow) {
+      ASSERT_TRUE(overflow) << "overflow expected for input " << input;
+    } else {
+      ASSERT_FALSE(overflow) << "overflow not expected for input " << input;
+    }
+  }
+}
+
+TEST_F(TestDecimalSql, ToInt64) {
+  // expected, input, overflow
+  std::vector<std::tuple<int64_t, DecimalScalar128, bool>> test_values = {
+      // simple ones
+      std::make_tuple(-16, DecimalScalar128{-16285, 38, 3}, false),
+      std::make_tuple(-162, DecimalScalar128{-16285, 38, 2}, false),
+      std::make_tuple(-1, DecimalScalar128{-16285, 38, 4}, false),
+
+      // border cases
+      std::make_tuple(INT64_MIN, DecimalScalar128{INT64_MIN, 38, 0}, false),
+      std::make_tuple(-32, DecimalScalar128{-32, 38, 0}, false),
+      std::make_tuple(0, DecimalScalar128{0, 0, 38, 0}, false),
+      std::make_tuple(32, DecimalScalar128{32, 38, 0}, false),
+      std::make_tuple(INT64_MAX, DecimalScalar128{INT64_MAX, 38, 0}, false),
+
+      // large scales
+      std::make_tuple(0, DecimalScalar128{123, 38, 16}, false),
+      std::make_tuple(0, DecimalScalar128{123, 38, 32}, false),
+      std::make_tuple(0, DecimalScalar128{123, 38, 38}, false),
+
+      // overflow test cases
+      // very large
+      std::make_tuple(0, DecimalScalar128{32768, 16, 38, 2}, true),
+      std::make_tuple(0, DecimalScalar128{INT64_MAX, UINT64_MAX, 38, 10}, true),
+      // very small
+      std::make_tuple(0, -DecimalScalar128{32768, 16, 38, 2}, true),
+      std::make_tuple(0, -DecimalScalar128{INT64_MAX, UINT64_MAX, 38, 10}, true),
+  };
+
+  for (auto iter : test_values) {
+    auto expected_value = std::get<0>(iter);
+    auto input = std::get<1>(iter);
+    auto expected_overflow = std::get<2>(iter);
+    bool overflow = false;
+
+    EXPECT_EQ(expected_value, decimalops::ToInt64(input, &overflow))
+        << "  failed on input " << input;
+    if (expected_overflow) {
+      ASSERT_TRUE(overflow) << "overflow expected for input " << input;
+    } else {
+      ASSERT_FALSE(overflow) << "overflow not expected for input " << input;
+    }
+  }
+}
+
 }  // namespace gandiva
diff --git a/cpp/src/gandiva/precompiled/decimal_wrapper.cc b/cpp/src/gandiva/precompiled/decimal_wrapper.cc
index 69a3b70..389bb6a 100644
--- a/cpp/src/gandiva/precompiled/decimal_wrapper.cc
+++ b/cpp/src/gandiva/precompiled/decimal_wrapper.cc
@@ -97,4 +97,130 @@ int32_t compare_internal_decimal128_decimal128(int64_t x_high, uint64_t x_low,
   return gandiva::decimalops::Compare(x, y);
 }
 
+FORCE_INLINE
+void abs_decimal128_internal(int64_t x_high, uint64_t x_low, int32_t x_precision,
+                             int32_t x_scale, int64_t* out_high, uint64_t* out_low) {
+  gandiva::BasicDecimal128 x(x_high, x_low);
+  x.Abs();
+  *out_high = x.high_bits();
+  *out_low = x.low_bits();
+}
+
+FORCE_INLINE
+void ceil_decimal128_internal(int64_t x_high, uint64_t x_low, int32_t x_precision,
+                              int32_t x_scale, int64_t* out_high, uint64_t* out_low) {
+  gandiva::BasicDecimalScalar128 x({x_high, x_low}, x_precision, x_scale);
+
+  bool overflow = false;
+  auto out = gandiva::decimalops::Ceil(x, &overflow);
+  *out_high = out.high_bits();
+  *out_low = out.low_bits();
+}
+
+FORCE_INLINE
+void floor_decimal128_internal(int64_t x_high, uint64_t x_low, int32_t x_precision,
+                               int32_t x_scale, int64_t* out_high, uint64_t* out_low) {
+  gandiva::BasicDecimalScalar128 x({x_high, x_low}, x_precision, x_scale);
+
+  bool overflow = false;
+  auto out = gandiva::decimalops::Floor(x, &overflow);
+  *out_high = out.high_bits();
+  *out_low = out.low_bits();
+}
+
+FORCE_INLINE
+void round_decimal128_internal(int64_t x_high, uint64_t x_low, int32_t x_precision,
+                               int32_t x_scale, int64_t* out_high, uint64_t* out_low) {
+  gandiva::BasicDecimalScalar128 x({x_high, x_low}, x_precision, x_scale);
+
+  bool overflow = false;
+  auto out = gandiva::decimalops::Round(x, 0, &overflow);
+  *out_high = out.high_bits();
+  *out_low = out.low_bits();
+}
+
+FORCE_INLINE
+void round_decimal128_int32_internal(int64_t x_high, uint64_t x_low, int32_t x_precision,
+                                     int32_t x_scale, int32_t rounding_scale,
+                                     int64_t* out_high, uint64_t* out_low) {
+  gandiva::BasicDecimalScalar128 x({x_high, x_low}, x_precision, x_scale);
+
+  bool overflow = false;
+  auto out = gandiva::decimalops::Round(x, rounding_scale, &overflow);
+  *out_high = out.high_bits();
+  *out_low = out.low_bits();
+}
+
+FORCE_INLINE
+void truncate_decimal128_internal(int64_t x_high, uint64_t x_low, int32_t x_precision,
+                                  int32_t x_scale, int64_t* out_high, uint64_t* out_low) {
+  gandiva::BasicDecimalScalar128 x({x_high, x_low}, x_precision, x_scale);
+
+  bool overflow = false;
+  auto out = gandiva::decimalops::Truncate(x, 0, &overflow);
+  *out_high = out.high_bits();
+  *out_low = out.low_bits();
+}
+
+FORCE_INLINE
+void truncate_decimal128_int32_internal(int64_t x_high, uint64_t x_low,
+                                        int32_t x_precision, int32_t x_scale,
+                                        int32_t rounding_scale, int64_t* out_high,
+                                        uint64_t* out_low) {
+  gandiva::BasicDecimalScalar128 x({x_high, x_low}, x_precision, x_scale);
+
+  bool overflow = false;
+  auto out = gandiva::decimalops::Truncate(x, rounding_scale, &overflow);
+  *out_high = out.high_bits();
+  *out_low = out.low_bits();
+}
+
+FORCE_INLINE
+double castFLOAT8_decimal128_internal(int64_t x_high, uint64_t x_low, int32_t x_precision,
+                                      int32_t x_scale) {
+  gandiva::BasicDecimalScalar128 x({x_high, x_low}, x_precision, x_scale);
+
+  bool overflow = false;
+  return gandiva::decimalops::ToDouble(x, &overflow);
+}
+
+FORCE_INLINE
+int64_t castBIGINT_decimal128_internal(int64_t x_high, uint64_t x_low,
+                                       int32_t x_precision, int32_t x_scale) {
+  gandiva::BasicDecimalScalar128 x({x_high, x_low}, x_precision, x_scale);
+
+  bool overflow = false;
+  return gandiva::decimalops::ToInt64(x, &overflow);
+}
+
+FORCE_INLINE
+void castDECIMAL_int64_internal(int64_t in, int32_t x_precision, int32_t x_scale,
+                                int64_t* out_high, uint64_t* out_low) {
+  bool overflow = false;
+  auto out = gandiva::decimalops::FromInt64(in, x_precision, x_scale, &overflow);
+  *out_high = out.high_bits();
+  *out_low = out.low_bits();
+}
+
+FORCE_INLINE
+void castDECIMAL_float64_internal(double in, int32_t x_precision, int32_t x_scale,
+                                  int64_t* out_high, uint64_t* out_low) {
+  bool overflow = false;
+  auto out = gandiva::decimalops::FromDouble(in, x_precision, x_scale, &overflow);
+  *out_high = out.high_bits();
+  *out_low = out.low_bits();
+}
+
+FORCE_INLINE
+void castDECIMAL_decimal128_internal(int64_t x_high, uint64_t x_low, int32_t x_precision,
+                                     int32_t x_scale, int32_t out_precision,
+                                     int32_t out_scale, int64_t* out_high,
+                                     int64_t* out_low) {
+  gandiva::BasicDecimalScalar128 x({x_high, x_low}, x_precision, x_scale);
+  bool overflow = false;
+  auto out = gandiva::decimalops::Convert(x, out_precision, out_scale, &overflow);
+  *out_high = out.high_bits();
+  *out_low = out.low_bits();
+}
+
 }  // extern "C"
diff --git a/cpp/src/gandiva/tests/decimal_test.cc b/cpp/src/gandiva/tests/decimal_test.cc
index 08435e4..a6a4c53 100644
--- a/cpp/src/gandiva/tests/decimal_test.cc
+++ b/cpp/src/gandiva/tests/decimal_test.cc
@@ -55,7 +55,12 @@ std::vector<Decimal128> TestDecimal::MakeDecimalVector(std::vector<std::string>
     DCHECK_OK(status);
 
     Decimal128 scaled_value;
-    status = str_value.Rescale(str_scale, scale, &scaled_value);
+    if (str_scale == scale) {
+      scaled_value = str_value;
+    } else {
+      status = str_value.Rescale(str_scale, scale, &scaled_value);
+      DCHECK_OK(status);
+    }
     ret.push_back(scaled_value);
   }
   return ret;
@@ -297,4 +302,182 @@ TEST_F(TestDecimal, TestCompare) {
                             outputs[5]);  // greater_than_or_equal_to
 }
 
+TEST_F(TestDecimal, TestRoundFunctions) {
+  // schema for input fields
+  constexpr int32_t precision = 38;
+  constexpr int32_t scale = 2;
+  auto decimal_type = std::make_shared<arrow::Decimal128Type>(precision, scale);
+  auto field_a = field("a", decimal_type);
+  auto schema = arrow::schema({field_a});
+
+  auto scale_1 = TreeExprBuilder::MakeLiteral(1);
+
+  // build expressions
+  auto exprs = std::vector<ExpressionPtr>{
+      TreeExprBuilder::MakeExpression("abs", {field_a}, field("abs_ceil", decimal_type)),
+      TreeExprBuilder::MakeExpression("ceil", {field_a},
+                                      field("res_ceil", arrow::decimal(precision, 0))),
+      TreeExprBuilder::MakeExpression("floor", {field_a},
+                                      field("res_floor", arrow::decimal(precision, 0))),
+      TreeExprBuilder::MakeExpression("round", {field_a},
+                                      field("res_round", arrow::decimal(precision, 0))),
+      TreeExprBuilder::MakeExpression(
+          "truncate", {field_a}, field("res_truncate", arrow::decimal(precision, 0))),
+
+      TreeExprBuilder::MakeExpression(
+          TreeExprBuilder::MakeFunction("round",
+                                        {TreeExprBuilder::MakeField(field_a), scale_1},
+                                        arrow::decimal(precision, 1)),
+          field("res_round_3", arrow::decimal(precision, 1))),
+
+      TreeExprBuilder::MakeExpression(
+          TreeExprBuilder::MakeFunction("truncate",
+                                        {TreeExprBuilder::MakeField(field_a), scale_1},
+                                        arrow::decimal(precision, 1)),
+          field("res_truncate_3", arrow::decimal(precision, 1))),
+  };
+
+  // Build a projector for the expression.
+  std::shared_ptr<Projector> projector;
+  auto status = Projector::Make(schema, exprs, TestConfiguration(), &projector);
+  DCHECK_OK(status);
+
+  // Create a row-batch with some sample data
+  int num_records = 4;
+  auto validity = {true, true, true, true};
+  auto array_a = MakeArrowArrayDecimal(
+      decimal_type, MakeDecimalVector({"1.23", "1.58", "-1.23", "-1.58"}, scale),
+      validity);
+
+  // prepare input record batch
+  auto in_batch = arrow::RecordBatch::Make(schema, num_records, {array_a});
+
+  // Evaluate expression
+  arrow::ArrayVector outputs;
+  status = projector->Evaluate(*in_batch, pool_, &outputs);
+  DCHECK_OK(status);
+
+  // Validate results
+
+  // abs(x)
+  EXPECT_ARROW_ARRAY_EQUALS(
+      MakeArrowArrayDecimal(decimal_type,
+                            MakeDecimalVector({"1.23", "1.58", "1.23", "1.58"}, scale),
+                            validity),
+      outputs[0]);
+
+  // ceil(x)
+  EXPECT_ARROW_ARRAY_EQUALS(
+      MakeArrowArrayDecimal(arrow::decimal(precision, 0),
+                            MakeDecimalVector({"2", "2", "-1", "-1"}, 0), validity),
+      outputs[1]);
+
+  // floor(x)
+  EXPECT_ARROW_ARRAY_EQUALS(
+      MakeArrowArrayDecimal(arrow::decimal(precision, 0),
+                            MakeDecimalVector({"1", "1", "-2", "-2"}, 0), validity),
+      outputs[2]);
+
+  // round(x)
+  EXPECT_ARROW_ARRAY_EQUALS(
+      MakeArrowArrayDecimal(arrow::decimal(precision, 0),
+                            MakeDecimalVector({"1", "2", "-1", "-2"}, 0), validity),
+      outputs[3]);
+
+  // truncate(x)
+  EXPECT_ARROW_ARRAY_EQUALS(
+      MakeArrowArrayDecimal(arrow::decimal(precision, 0),
+                            MakeDecimalVector({"1", "1", "-1", "-1"}, 0), validity),
+      outputs[4]);
+
+  // round(x, 1)
+  EXPECT_ARROW_ARRAY_EQUALS(
+      MakeArrowArrayDecimal(arrow::decimal(precision, 1),
+                            MakeDecimalVector({"1.2", "1.6", "-1.2", "-1.6"}, 1),
+                            validity),
+      outputs[5]);
+
+  // truncate(x, 1)
+  EXPECT_ARROW_ARRAY_EQUALS(
+      MakeArrowArrayDecimal(arrow::decimal(precision, 1),
+                            MakeDecimalVector({"1.2", "1.5", "-1.2", "-1.5"}, 1),
+                            validity),
+      outputs[6]);
+}
+
+TEST_F(TestDecimal, TestCastFunctions) {
+  // schema for input fields
+  constexpr int32_t precision = 38;
+  constexpr int32_t scale = 2;
+  auto decimal_type = std::make_shared<arrow::Decimal128Type>(precision, scale);
+  auto decimal_type_scale_1 = std::make_shared<arrow::Decimal128Type>(precision, 1);
+  auto field_int64 = field("intt64", arrow::int64());
+  auto field_float64 = field("float64", arrow::float64());
+  auto field_dec = field("dec", decimal_type);
+  auto schema = arrow::schema({field_int64, field_float64, field_dec});
+
+  // build expressions
+  auto exprs = std::vector<ExpressionPtr>{
+      TreeExprBuilder::MakeExpression("castDECIMAL", {field_int64},
+                                      field("int64_to_dec", decimal_type)),
+      TreeExprBuilder::MakeExpression("castDECIMAL", {field_float64},
+                                      field("float64_to_dec", decimal_type)),
+      TreeExprBuilder::MakeExpression("castDECIMAL", {field_dec},
+                                      field("dec_to_dec", decimal_type_scale_1)),
+      TreeExprBuilder::MakeExpression("castBIGINT", {field_dec},
+                                      field("dec_to_int64", arrow::int64())),
+      TreeExprBuilder::MakeExpression("castFLOAT8", {field_dec},
+                                      field("dec_to_float64", arrow::float64())),
+  };
+
+  // Build a projector for the expression.
+  std::shared_ptr<Projector> projector;
+  auto status = Projector::Make(schema, exprs, TestConfiguration(), &projector);
+  DCHECK_OK(status);
+
+  // Create a row-batch with some sample data
+  int num_records = 4;
+  auto validity = {true, true, true, true};
+
+  auto array_int64 = MakeArrowArrayInt64({123, 158, -123, -158});
+  auto array_float64 = MakeArrowArrayFloat64({1.23, 1.58, -1.23, -1.58});
+  auto array_dec = MakeArrowArrayDecimal(
+      decimal_type, MakeDecimalVector({"1.23", "1.58", "-1.23", "-1.58"}, scale),
+      validity);
+
+  // prepare input record batch
+  auto in_batch = arrow::RecordBatch::Make(schema, num_records,
+                                           {array_int64, array_float64, array_dec});
+
+  // Evaluate expression
+  arrow::ArrayVector outputs;
+  status = projector->Evaluate(*in_batch, pool_, &outputs);
+  DCHECK_OK(status);
+
+  // Validate results
+
+  // castDECIMAL(int64)
+  EXPECT_ARROW_ARRAY_EQUALS(
+      MakeArrowArrayDecimal(decimal_type,
+                            MakeDecimalVector({"123", "158", "-123", "-158"}, scale),
+                            validity),
+      outputs[0]);
+
+  // castDECIMAL(float64)
+  EXPECT_ARROW_ARRAY_EQUALS(array_dec, outputs[1]);
+
+  // castDECIMAL(decimal)
+  EXPECT_ARROW_ARRAY_EQUALS(
+      MakeArrowArrayDecimal(arrow::decimal(precision, 1),
+                            MakeDecimalVector({"1.2", "1.6", "-1.2", "-1.6"}, 1),
+                            validity),
+      outputs[2]);
+
+  // castBIGINT(decimal)
+  EXPECT_ARROW_ARRAY_EQUALS(MakeArrowArrayInt64({1, 1, -1, -1}), outputs[3]);
+
+  // castDOUBLE(decimal)
+  EXPECT_ARROW_ARRAY_EQUALS(array_float64, outputs[4]);
+}
+
 }  // namespace gandiva