You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ap...@apache.org on 2015/08/13 20:02:16 UTC

hive git commit: HIVE-11103 Add banker's rounding BROUND UDF (Alexander Pivovarov, reviewed by Jason Dere)

Repository: hive
Updated Branches:
  refs/heads/master 139101d6c -> bd90fc349


HIVE-11103 Add banker's rounding BROUND UDF (Alexander Pivovarov, reviewed by Jason Dere)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/bd90fc34
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bd90fc34
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bd90fc34

Branch: refs/heads/master
Commit: bd90fc349fcd171a58f583928408d029af9dbf4e
Parents: 139101d
Author: Alexander Pivovarov <ap...@gmail.com>
Authored: Sun Jul 26 21:43:31 2015 -0700
Committer: Alexander Pivovarov <ap...@gmail.com>
Committed: Thu Aug 13 10:59:01 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ant/GenVectorCode.java   |   2 +
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   1 +
 .../BRoundWithNumDigitsDoubleToDouble.java      |  42 ++++
 .../ql/exec/vector/expressions/DecimalUtil.java |  18 ++
 ...FuncBRoundWithNumDigitsDecimalToDecimal.java |  40 ++++
 .../FuncRoundWithNumDigitsDecimalToDecimal.java |  14 +-
 .../ql/exec/vector/expressions/MathExpr.java    |  22 ++
 .../hive/ql/optimizer/physical/Vectorizer.java  |   1 +
 .../hive/ql/udf/generic/GenericUDFBRound.java   |  68 +++++++
 .../hive/ql/udf/generic/GenericUDFRound.java    |  41 ++--
 .../hadoop/hive/ql/udf/generic/RoundUtils.java  |  14 ++
 .../exec/vector/TestVectorizationContext.java   |  15 ++
 .../ql/udf/generic/TestGenericUDFBRound.java    | 202 +++++++++++++++++++
 ql/src/test/queries/clientpositive/udf_bround.q |  44 ++++
 .../test/queries/clientpositive/vector_bround.q |  14 ++
 .../results/clientpositive/show_functions.q.out |   1 +
 .../results/clientpositive/udf_bround.q.out     | 119 +++++++++++
 .../results/clientpositive/vector_bround.q.out  |  86 ++++++++
 .../hadoop/hive/common/type/HiveDecimal.java    |   1 +
 19 files changed, 727 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
----------------------------------------------------------------------
diff --git a/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java b/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
index 54f3783..fede273 100644
--- a/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
+++ b/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
@@ -696,6 +696,7 @@ public class GenVectorCode extends Task {
       // template, <ClassNamePrefix>, <ReturnType>, <OperandType>, <FuncName>, <OperandCast>,
       //   <ResultCast>, <Cleanup> <VectorExprArgType>
       {"ColumnUnaryFunc", "FuncRound", "double", "double", "MathExpr.round", "", "", "", ""},
+      {"ColumnUnaryFunc", "FuncBRound", "double", "double", "MathExpr.bround", "", "", "", ""},
       // round(longCol) returns a long and is a no-op. So it will not be implemented here.
       // round(Col, N) is a special case and will be implemented separately from this template
       {"ColumnUnaryFunc", "FuncFloor", "long", "double", "Math.floor", "", "(long)", "", ""},
@@ -752,6 +753,7 @@ public class GenVectorCode extends Task {
       {"DecimalColumnUnaryFunc", "FuncAbs", "decimal", "DecimalUtil.abs"},
       {"DecimalColumnUnaryFunc", "FuncSign", "long", "DecimalUtil.sign"},
       {"DecimalColumnUnaryFunc", "FuncRound", "decimal", "DecimalUtil.round"},
+      {"DecimalColumnUnaryFunc", "FuncBRound", "decimal", "DecimalUtil.bround"},
       {"DecimalColumnUnaryFunc", "FuncNegate", "decimal", "DecimalUtil.negate"},
 
       // Casts

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index fb06d44..9edcc4d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -192,6 +192,7 @@ public final class FunctionRegistry {
     system.registerGenericUDF("size", GenericUDFSize.class);
 
     system.registerGenericUDF("round", GenericUDFRound.class);
+    system.registerGenericUDF("bround", GenericUDFBRound.class);
     system.registerGenericUDF("floor", GenericUDFFloor.class);
     system.registerUDF("sqrt", UDFSqrt.class, false);
     system.registerGenericUDF("cbrt", GenericUDFCbrt.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/BRoundWithNumDigitsDoubleToDouble.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/BRoundWithNumDigitsDoubleToDouble.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/BRoundWithNumDigitsDoubleToDouble.java
new file mode 100644
index 0000000..0a49e45
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/BRoundWithNumDigitsDoubleToDouble.java
@@ -0,0 +1,42 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.udf.generic.RoundUtils;
+
+// Vectorized implementation of BROUND(Col, N) function
+public class BRoundWithNumDigitsDoubleToDouble extends RoundWithNumDigitsDoubleToDouble
+    implements ISetLongArg {
+  private static final long serialVersionUID = 18493485928L;
+
+  public BRoundWithNumDigitsDoubleToDouble(int colNum, long scalarVal, int outputColumn) {
+    super(colNum, scalarVal, outputColumn);
+  }
+
+  public BRoundWithNumDigitsDoubleToDouble() {
+    super();
+  }
+
+  // Round to the specified number of decimal places using half-even round function.
+  @Override
+  public double func(double d) {
+    return RoundUtils.bround(d, getDecimalPlaces().get());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalUtil.java
index ef80059..a01f7a2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalUtil.java
@@ -314,6 +314,24 @@ public class DecimalUtil {
     }
   }
 
+  public static void bround(int i, HiveDecimalWritable input, int decimalPlaces, DecimalColumnVector outputColVector) {
+    try {
+      outputColVector.set(i, RoundUtils.bround(input.getHiveDecimal(), decimalPlaces));
+    } catch (ArithmeticException e) {
+      outputColVector.noNulls = false;
+      outputColVector.isNull[i] = true;
+    }
+  }
+
+  public static void bround(int i, HiveDecimalWritable input, DecimalColumnVector outputColVector) {
+    try {
+      outputColVector.set(i, RoundUtils.bround(input.getHiveDecimal(), outputColVector.scale));
+    } catch (ArithmeticException e) {
+      outputColVector.noNulls = false;
+      outputColVector.isNull[i] = true;
+    }
+  }
+
   public static void sign(int i, HiveDecimal input, LongColumnVector outputColVector) {
     outputColVector.vector[i] = input.signum();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBRoundWithNumDigitsDecimalToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBRoundWithNumDigitsDecimalToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBRoundWithNumDigitsDecimalToDecimal.java
new file mode 100644
index 0000000..e174575
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncBRoundWithNumDigitsDecimalToDecimal.java
@@ -0,0 +1,40 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+
+public class FuncBRoundWithNumDigitsDecimalToDecimal extends FuncRoundWithNumDigitsDecimalToDecimal {
+  private static final long serialVersionUID = 1865384957262L;
+
+  public FuncBRoundWithNumDigitsDecimalToDecimal(int colNum, int scalarValue, int outputColumn) {
+    super(colNum, scalarValue, outputColumn);
+  }
+
+  public FuncBRoundWithNumDigitsDecimalToDecimal() {
+    super();
+  }
+
+  @Override
+  protected void round(int i, HiveDecimalWritable input, int decimalPlaces,
+      DecimalColumnVector outputColVector) {
+    DecimalUtil.bround(i, input, decimalPlaces, outputColVector);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRoundWithNumDigitsDecimalToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRoundWithNumDigitsDecimalToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRoundWithNumDigitsDecimalToDecimal.java
index 9f3e8a3..a18bb55 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRoundWithNumDigitsDecimalToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRoundWithNumDigitsDecimalToDecimal.java
@@ -73,7 +73,7 @@ public class FuncRoundWithNumDigitsDecimalToDecimal extends VectorExpression {
       // All must be selected otherwise size would be zero
       // Repeating property will not change.
       outputIsNull[0] = inputIsNull[0];
-      DecimalUtil.round(0, vector[0], decimalPlaces, outputColVector);
+      round(0, vector[0], decimalPlaces, outputColVector);
       outputColVector.isRepeating = true;
     } else if (inputColVector.noNulls) {
       if (batch.selectedInUse) {
@@ -82,14 +82,14 @@ public class FuncRoundWithNumDigitsDecimalToDecimal extends VectorExpression {
 
           // Set isNull because decimal operation can yield a null.
           outputIsNull[i] = false;
-          DecimalUtil.round(i, vector[i], decimalPlaces, outputColVector);
+          round(i, vector[i], decimalPlaces, outputColVector);
         }
       } else {
 
         // Set isNull because decimal operation can yield a null.
         Arrays.fill(outputIsNull, 0, n, false);
         for(int i = 0; i != n; i++) {
-          DecimalUtil.round(i, vector[i], decimalPlaces, outputColVector);
+          round(i, vector[i], decimalPlaces, outputColVector);
         }
       }
       outputColVector.isRepeating = false;
@@ -98,12 +98,12 @@ public class FuncRoundWithNumDigitsDecimalToDecimal extends VectorExpression {
         for(int j = 0; j != n; j++) {
           int i = sel[j];
           outputIsNull[i] = inputIsNull[i];
-          DecimalUtil.round(i, vector[i], decimalPlaces, outputColVector);
+          round(i, vector[i], decimalPlaces, outputColVector);
         }
       } else {
         System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
         for(int i = 0; i != n; i++) {
-          DecimalUtil.round(i, vector[i], decimalPlaces, outputColVector);
+          round(i, vector[i], decimalPlaces, outputColVector);
         }
       }
       outputColVector.isRepeating = false;
@@ -133,4 +133,8 @@ public class FuncRoundWithNumDigitsDecimalToDecimal extends VectorExpression {
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
   }
+
+  protected void round(int i, HiveDecimalWritable input, int decimalPlaces, DecimalColumnVector outputColVector) {
+    DecimalUtil.round(i, input, decimalPlaces, outputColVector);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java
index aef923e..67bf567 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/MathExpr.java
@@ -35,6 +35,28 @@ public class MathExpr {
     }
   }
 
+  // Round using the "half-even" method used in Hive.
+  public static double bround(double d) {
+    long intPart = (long) (d);
+    if (d > 0.0) {
+      if (d - intPart == 0.5d) {
+        if (intPart % 2 == 0) {
+          return intPart;
+        }
+        return intPart + 1;
+      }
+      return (double) ((long) (d + 0.5d));
+    } else {
+      if (intPart - d == 0.5d) {
+        if (intPart % 2 == 0) {
+          return intPart;
+        }
+        return intPart - 1;
+      }
+      return (double) ((long) (d - 0.5d));
+    }
+  }
+
   public static double log2(double d) {
     return Math.log(d) / Math.log(2);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index 82c3e50..7ecd50a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -250,6 +250,7 @@ public class Vectorizer implements PhysicalPlanResolver {
     supportedGenericUDFs.add(UDFLog.class);
     supportedGenericUDFs.add(GenericUDFPower.class);
     supportedGenericUDFs.add(GenericUDFRound.class);
+    supportedGenericUDFs.add(GenericUDFBRound.class);
     supportedGenericUDFs.add(GenericUDFPosMod.class);
     supportedGenericUDFs.add(UDFSqrt.class);
     supportedGenericUDFs.add(UDFSign.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBRound.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBRound.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBRound.java
new file mode 100644
index 0000000..4a59eb3
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBRound.java
@@ -0,0 +1,68 @@
+/**
+ * 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.hadoop.hive.ql.udf.generic;
+
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.BRoundWithNumDigitsDoubleToDouble;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.FuncBRoundWithNumDigitsDecimalToDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncBRoundDecimalToDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncBRoundDoubleToDouble;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+
+@Description(name = "bround",
+value = "_FUNC_(x[, d]) - round x to d decimal places using HALF_EVEN rounding mode.",
+extended = "Banker's rounding. The value is rounded to the nearest even number. Also known as Gaussian rounding.\n"
+  + "Example:\n"
+  + "  > SELECT _FUNC_(12.25, 1);\n  12.2")
+@VectorizedExpressions({ FuncBRoundDoubleToDouble.class, BRoundWithNumDigitsDoubleToDouble.class,
+    FuncBRoundWithNumDigitsDecimalToDecimal.class, FuncBRoundDecimalToDecimal.class })
+public class GenericUDFBRound extends GenericUDFRound {
+
+  @Override
+  protected HiveDecimal round(HiveDecimal input, int scale) {
+    return RoundUtils.bround(input, scale);
+  }
+
+  @Override
+  protected long round(long input, int scale) {
+    return RoundUtils.bround(input, scale);
+  }
+
+  @Override
+  protected double round(double input, int scale) {
+    return RoundUtils.bround(input, scale);
+  }
+
+  @Override
+  protected DoubleWritable round(DoubleWritable input, int scale) {
+    double d = input.get();
+    if (Double.isNaN(d) || Double.isInfinite(d)) {
+      return new DoubleWritable(d);
+    } else {
+      return new DoubleWritable(RoundUtils.bround(d, scale));
+    }
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    return getStandardDisplayString("bround", children);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRound.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRound.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRound.java
index 963e4a8..ae81fe3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRound.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRound.java
@@ -101,34 +101,37 @@ public class GenericUDFRound extends GenericUDF {
         break;
       case BYTE:
         if (!(scaleOI instanceof WritableConstantByteObjectInspector)) {
-          throw new UDFArgumentTypeException(1, "ROUND second argument only takes constant");
+          throw new UDFArgumentTypeException(1, getFuncName().toUpperCase() + " second argument only takes constant");
         }
         scale = ((WritableConstantByteObjectInspector)scaleOI).getWritableConstantValue().get();
         break;
       case SHORT:
         if (!(scaleOI instanceof WritableConstantShortObjectInspector)) {
-          throw new UDFArgumentTypeException(1, "ROUND second argument only takes constant");
+          throw new UDFArgumentTypeException(1, getFuncName().toUpperCase() + " second argument only takes constant");
         }
         scale = ((WritableConstantShortObjectInspector)scaleOI).getWritableConstantValue().get();
         break;
       case INT:
         if (!(scaleOI instanceof WritableConstantIntObjectInspector)) {
-          throw new UDFArgumentTypeException(1, "ROUND second argument only takes constant");
+          throw new UDFArgumentTypeException(1, getFuncName().toUpperCase() + " second argument only takes constant");
         }
         scale = ((WritableConstantIntObjectInspector)scaleOI).getWritableConstantValue().get();
         break;
       case LONG:
         if (!(scaleOI instanceof WritableConstantLongObjectInspector)) {
-          throw new UDFArgumentTypeException(1, "ROUND second argument only takes constant");
+          throw new UDFArgumentTypeException(1, getFuncName().toUpperCase()
+              + " second argument only takes constant");
         }
         long l = ((WritableConstantLongObjectInspector)scaleOI).getWritableConstantValue().get();
         if (l < Integer.MIN_VALUE || l > Integer.MAX_VALUE) {
-          throw new UDFArgumentException("ROUND scale argument out of allowed range");
+          throw new UDFArgumentException(getFuncName().toUpperCase()
+              + " scale argument out of allowed range");
         }
         scale = (int)l;
         break;
       default:
-        throw new UDFArgumentTypeException(1, "ROUND second argument only takes integer constant");
+        throw new UDFArgumentTypeException(1, getFuncName().toUpperCase()
+            + " second argument only takes integer constant");
       }
     }
 
@@ -199,7 +202,7 @@ public class GenericUDFRound extends GenericUDF {
       return null;
     case DECIMAL:
       HiveDecimalWritable decimalWritable = (HiveDecimalWritable) inputOI.getPrimitiveWritableObject(input);
-      HiveDecimal dec = RoundUtils.round(decimalWritable.getHiveDecimal(), scale);
+      HiveDecimal dec = round(decimalWritable.getHiveDecimal(), scale);
       if (dec == null) {
         return null;
       }
@@ -209,32 +212,32 @@ public class GenericUDFRound extends GenericUDF {
       if (scale >= 0) {
         return byteWritable;
       } else {
-        return new ByteWritable((byte)RoundUtils.round(byteWritable.get(), scale));
+        return new ByteWritable((byte)round(byteWritable.get(), scale));
       }
     case SHORT:
       ShortWritable shortWritable = (ShortWritable)inputOI.getPrimitiveWritableObject(input);
       if (scale >= 0) {
         return shortWritable;
       } else {
-        return new ShortWritable((short)RoundUtils.round(shortWritable.get(), scale));
+        return new ShortWritable((short)round(shortWritable.get(), scale));
       }
     case INT:
       IntWritable intWritable = (IntWritable)inputOI.getPrimitiveWritableObject(input);
       if (scale >= 0) {
         return intWritable;
       } else {
-        return new IntWritable((int)RoundUtils.round(intWritable.get(), scale));
+        return new IntWritable((int)round(intWritable.get(), scale));
       }
     case LONG:
       LongWritable longWritable = (LongWritable)inputOI.getPrimitiveWritableObject(input);
       if (scale >= 0) {
         return longWritable;
       } else {
-        return new LongWritable(RoundUtils.round(longWritable.get(), scale));
+        return new LongWritable(round(longWritable.get(), scale));
       }
     case FLOAT:
       float f = ((FloatWritable)inputOI.getPrimitiveWritableObject(input)).get();
-      return new FloatWritable((float)RoundUtils.round(f, scale));
+      return new FloatWritable((float)round(f, scale));
      case DOUBLE:
        return round(((DoubleWritable)inputOI.getPrimitiveWritableObject(input)), scale);
     case STRING:
@@ -252,7 +255,19 @@ public class GenericUDFRound extends GenericUDF {
     }
   }
 
-  private static DoubleWritable round(DoubleWritable input, int scale) {
+  protected HiveDecimal round(HiveDecimal input, int scale) {
+    return RoundUtils.round(input, scale);
+  }
+
+  protected long round(long input, int scale) {
+    return RoundUtils.round(input, scale);
+  }
+
+  protected double round(double input, int scale) {
+    return RoundUtils.round(input, scale);
+  }
+
+  protected DoubleWritable round(DoubleWritable input, int scale) {
     double d = input.get();
     if (Double.isNaN(d) || Double.isInfinite(d)) {
       return new DoubleWritable(d);

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/RoundUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/RoundUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/RoundUtils.java
index 0b389a5..7fd1641 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/RoundUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/RoundUtils.java
@@ -49,12 +49,26 @@ public class RoundUtils {
     return BigDecimal.valueOf(input).setScale(scale, RoundingMode.HALF_UP).doubleValue();
   }
 
+  public static double bround(double input, int scale) {
+    if (Double.isNaN(input) || Double.isInfinite(input)) {
+      return input;
+    }
+    return BigDecimal.valueOf(input).setScale(scale, RoundingMode.HALF_EVEN).doubleValue();
+  }
+
   public static long round(long input, int scale) {
     return BigDecimal.valueOf(input).setScale(scale, RoundingMode.HALF_UP).longValue();
   }
 
+  public static long bround(long input, int scale) {
+    return BigDecimal.valueOf(input).setScale(scale, RoundingMode.HALF_EVEN).longValue();
+  }
+
   public static HiveDecimal round(HiveDecimal input, int scale) {
     return input.setScale(scale, HiveDecimal.ROUND_HALF_UP);
   }
 
+  public static HiveDecimal bround(HiveDecimal input, int scale) {
+    return input.setScale(scale, HiveDecimal.ROUND_HALF_EVEN);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
index 98a8c3e..8470c47 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
@@ -32,6 +32,7 @@ import junit.framework.Assert;
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.BRoundWithNumDigitsDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.ColAndCol;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.ColOrCol;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DoubleColumnInList;
@@ -94,6 +95,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterVarCharColumn
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterVarCharColumnNotBetween;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterCharColumnBetween;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterCharColumnNotBetween;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncBRoundDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncLnDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncRoundDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncSinDoubleToDouble;
@@ -114,6 +116,7 @@ import org.apache.hadoop.hive.ql.udf.UDFLog;
 import org.apache.hadoop.hive.ql.udf.UDFSin;
 import org.apache.hadoop.hive.ql.udf.UDFYear;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBRound;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIf;
@@ -989,6 +992,12 @@ public class TestVectorizationContext {
     ve = vc.getVectorExpression(mathFuncExpr);
     Assert.assertEquals(FuncRoundDoubleToDouble.class, ve.getClass());
 
+    // BRound without digits
+    GenericUDFBRound udfBRound = new GenericUDFBRound();
+    mathFuncExpr.setGenericUDF(udfBRound);
+    ve = vc.getVectorExpression(mathFuncExpr);
+    Assert.assertEquals(FuncBRoundDoubleToDouble.class, ve.getClass());
+
     // Round with digits
     mathFuncExpr.setGenericUDF(udfRound);
     children2.add(new ExprNodeConstantDesc(4));
@@ -997,6 +1006,12 @@ public class TestVectorizationContext {
     Assert.assertEquals(RoundWithNumDigitsDoubleToDouble.class, ve.getClass());
     Assert.assertEquals(4, ((RoundWithNumDigitsDoubleToDouble) ve).getDecimalPlaces().get());
 
+    // BRound with digits
+    mathFuncExpr.setGenericUDF(udfBRound);
+    ve = vc.getVectorExpression(mathFuncExpr);
+    Assert.assertEquals(BRoundWithNumDigitsDoubleToDouble.class, ve.getClass());
+    Assert.assertEquals(4, ((BRoundWithNumDigitsDoubleToDouble) ve).getDecimalPlaces().get());
+
     // Log with int base
     gudfBridge = new GenericUDFBridge("log", false, UDFLog.class.getName());
     mathFuncExpr.setGenericUDF(gudfBridge);

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFBRound.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFBRound.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFBRound.java
new file mode 100644
index 0000000..d526dd8
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFBRound.java
@@ -0,0 +1,202 @@
+/**
+ * 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.hadoop.hive.ql.udf.generic;
+
+import static java.math.BigDecimal.ROUND_HALF_EVEN;
+
+import java.math.BigDecimal;
+
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.MathExpr;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestGenericUDFBRound {
+
+  @Test
+  public void testDouble() throws HiveException {
+    GenericUDFBRound udf = new GenericUDFBRound();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+
+    IntWritable scale = new IntWritable(0);
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.intTypeInfo, scale);
+
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runDouble(2.5, scale, 2.0, udf);
+    runDouble(3.5, scale, 4.0, udf);
+
+    runDouble(2.49, scale, 2.0, udf);
+    runDouble(3.49, scale, 3.0, udf);
+
+    runDouble(2.51, scale, 3.0, udf);
+    runDouble(3.51, scale, 4.0, udf);
+
+    runDouble(2.4, scale, 2.0, udf);
+    runDouble(3.4, scale, 3.0, udf);
+
+    runDouble(2.6, scale, 3.0, udf);
+    runDouble(3.6, scale, 4.0, udf);
+  }
+
+  @Test
+  public void testDoubleScaleMinus1() throws HiveException {
+    GenericUDFBRound udf = new GenericUDFBRound();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+
+    IntWritable scale = new IntWritable(-1);
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.intTypeInfo, scale);
+
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runDouble(55.0, scale, 60.0, udf);
+    runDouble(45.0, scale, 40.0, udf);
+
+    runDouble(54.9, scale, 50.0, udf);
+    runDouble(44.9, scale, 40.0, udf);
+
+    runDouble(55.1, scale, 60.0, udf);
+    runDouble(45.1, scale, 50.0, udf);
+
+    runDouble(-55.0, scale, -60.0, udf);
+    runDouble(-45.0, scale, -40.0, udf);
+
+    runDouble(-54.9, scale, -50.0, udf);
+    runDouble(-44.9, scale, -40.0, udf);
+
+    runDouble(-55.1, scale, -60.0, udf);
+    runDouble(-45.1, scale, -50.0, udf);
+  }
+
+  @Test
+  public void testFloat() throws HiveException {
+    GenericUDFBRound udf = new GenericUDFBRound();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableFloatObjectInspector;
+
+    IntWritable scale = new IntWritable(0);
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.intTypeInfo, scale);
+
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runFloat(2.5f, scale, 2.0f, udf);
+    runFloat(3.5f, scale, 4.0f, udf);
+
+    runFloat(2.49f, scale, 2.0f, udf);
+    runFloat(3.49f, scale, 3.0f, udf);
+
+    runFloat(2.51f, scale, 3.0f, udf);
+    runFloat(3.51f, scale, 4.0f, udf);
+
+    runFloat(2.4f, scale, 2.0f, udf);
+    runFloat(3.4f, scale, 3.0f, udf);
+
+    runFloat(2.6f, scale, 3.0f, udf);
+    runFloat(3.6f, scale, 4.0f, udf);
+  }
+
+  @Test
+  public void testDecimal() throws HiveException {
+    GenericUDFBRound udf = new GenericUDFBRound();
+    ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector;
+
+    IntWritable scale = new IntWritable(0);
+    ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory
+        .getPrimitiveWritableConstantObjectInspector(TypeInfoFactory.intTypeInfo, scale);
+
+    ObjectInspector[] arguments = { valueOI0, valueOI1 };
+
+    udf.initialize(arguments);
+
+    runDecimal(2.5, scale, 2.0, udf);
+    runDecimal(3.5, scale, 4.0, udf);
+
+    runDecimal(2.49, scale, 2.0, udf);
+    runDecimal(3.49, scale, 3.0, udf);
+
+    runDecimal(2.51, scale, 3.0, udf);
+    runDecimal(3.51, scale, 4.0, udf);
+
+    runDecimal(2.4, scale, 2.0, udf);
+    runDecimal(3.4, scale, 3.0, udf);
+
+    runDecimal(2.6, scale, 3.0, udf);
+    runDecimal(3.6, scale, 4.0, udf);
+  }
+
+  @Test
+  public void testMathExprBround() throws HiveException {
+    double[] vArr = { 1.5, 2.5, -1.5, -2.5, 1.49, 1.51 };
+    for (double v : vArr) {
+      double v1 = RoundUtils.bround(v, 0);
+      double v2 = MathExpr.bround(v);
+      Assert.assertEquals(v1, v2, 0.00001);
+
+      double v3 = BigDecimal.valueOf(v).setScale(0, ROUND_HALF_EVEN).doubleValue();
+      Assert.assertEquals(v3, v2, 0.00001);
+    }
+  }
+
+  private void runDouble(double v, IntWritable scale, Double expV, GenericUDF udf)
+      throws HiveException {
+    DeferredObject valueObj0 = new DeferredJavaObject(new DoubleWritable(v));
+    DeferredObject valueObj1 = new DeferredJavaObject(scale);
+    DeferredObject[] args = { valueObj0, valueObj1 };
+    DoubleWritable output = (DoubleWritable) udf.evaluate(args);
+    Assert.assertEquals("bround() test ", expV.doubleValue(), output.get(), 0.00001);
+  }
+
+  private void runFloat(float v, IntWritable scale, Float expV, GenericUDF udf)
+      throws HiveException {
+    DeferredObject valueObj0 = new DeferredJavaObject(new FloatWritable(v));
+    DeferredObject valueObj1 = new DeferredJavaObject(scale);
+    DeferredObject[] args = { valueObj0, valueObj1 };
+    FloatWritable output = (FloatWritable) udf.evaluate(args);
+    Assert.assertEquals("bround() test ", expV.floatValue(), output.get(), 0.001f);
+  }
+
+  private void runDecimal(double v, IntWritable scale, Double expV, GenericUDF udf)
+      throws HiveException {
+    HiveDecimal hd = HiveDecimal.create(BigDecimal.valueOf(v));
+    DeferredObject valueObj0 = new DeferredJavaObject(new HiveDecimalWritable(hd));
+    DeferredObject valueObj1 = new DeferredJavaObject(scale);
+    DeferredObject[] args = { valueObj0, valueObj1 };
+    HiveDecimalWritable output = (HiveDecimalWritable) udf.evaluate(args);
+    Assert.assertEquals("bround() test ", expV.doubleValue(),
+        output.getHiveDecimal().doubleValue(), 0.00001);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/test/queries/clientpositive/udf_bround.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udf_bround.q b/ql/src/test/queries/clientpositive/udf_bround.q
new file mode 100644
index 0000000..ef2c33a
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/udf_bround.q
@@ -0,0 +1,44 @@
+set hive.fetch.task.conversion=more;
+
+DESCRIBE FUNCTION bround;
+DESC FUNCTION EXTENDED bround;
+
+select
+bround(2.5),
+bround(3.5),
+bround(2.49),
+bround(3.49),
+bround(2.51),
+bround(3.51);
+
+select
+bround(1.25, 1),
+bround(1.35, 1),
+bround(1.249, 1),
+bround(1.349, 1),
+bround(1.251, 1),
+bround(1.351, 1);
+
+select
+bround(-1.25, 1),
+bround(-1.35, 1),
+bround(-1.249, 1),
+bround(-1.349, 1),
+bround(-1.251, 1),
+bround(-1.351, 1);
+
+select
+bround(55.0, -1),
+bround(45.0, -1),
+bround(54.9, -1),
+bround(44.9, -1),
+bround(55.1, -1),
+bround(45.1, -1);
+
+select
+bround(-55.0, -1),
+bround(-45.0, -1),
+bround(-54.9, -1),
+bround(-44.9, -1),
+bround(-55.1, -1),
+bround(-45.1, -1);

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/test/queries/clientpositive/vector_bround.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_bround.q b/ql/src/test/queries/clientpositive/vector_bround.q
new file mode 100644
index 0000000..380d51c
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_bround.q
@@ -0,0 +1,14 @@
+create table test_vector_bround(v0 double, v1 double) stored as orc;
+insert into table test_vector_bround
+values
+(2.5, 1.25),
+(3.5, 1.35),
+(-2.5, -1.25),
+(-3.5, -1.35),
+(2.49, 1.249),
+(3.49, 1.349),
+(2.51, 1.251),
+(3.51, 1.351);
+set hive.vectorized.execution.enabled=true;
+explain select bround(v0), bround(v1, 1) from test_vector_bround;
+select bround(v0), bround(v1, 1) from test_vector_bround;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/test/results/clientpositive/show_functions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_functions.q.out b/ql/src/test/results/clientpositive/show_functions.q.out
index 5de4ffc..540079b 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -33,6 +33,7 @@ avg
 base64
 between
 bin
+bround
 case
 cbrt
 ceil

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/test/results/clientpositive/udf_bround.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_bround.q.out b/ql/src/test/results/clientpositive/udf_bround.q.out
new file mode 100644
index 0000000..4dcea6d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/udf_bround.q.out
@@ -0,0 +1,119 @@
+PREHOOK: query: DESCRIBE FUNCTION bround
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESCRIBE FUNCTION bround
+POSTHOOK: type: DESCFUNCTION
+bround(x[, d]) - round x to d decimal places using HALF_EVEN rounding mode.
+PREHOOK: query: DESC FUNCTION EXTENDED bround
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESC FUNCTION EXTENDED bround
+POSTHOOK: type: DESCFUNCTION
+bround(x[, d]) - round x to d decimal places using HALF_EVEN rounding mode.
+Banker's rounding. The value is rounded to the nearest even number. Also known as Gaussian rounding.
+Example:
+  > SELECT bround(12.25, 1);
+  12.2
+PREHOOK: query: select
+bround(2.5),
+bround(3.5),
+bround(2.49),
+bround(3.49),
+bround(2.51),
+bround(3.51)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+bround(2.5),
+bround(3.5),
+bround(2.49),
+bround(3.49),
+bround(2.51),
+bround(3.51)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+2.0	4.0	2.0	3.0	3.0	4.0
+PREHOOK: query: select
+bround(1.25, 1),
+bround(1.35, 1),
+bround(1.249, 1),
+bround(1.349, 1),
+bround(1.251, 1),
+bround(1.351, 1)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+bround(1.25, 1),
+bround(1.35, 1),
+bround(1.249, 1),
+bround(1.349, 1),
+bround(1.251, 1),
+bround(1.351, 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+1.2	1.4	1.2	1.3	1.3	1.4
+PREHOOK: query: select
+bround(-1.25, 1),
+bround(-1.35, 1),
+bround(-1.249, 1),
+bround(-1.349, 1),
+bround(-1.251, 1),
+bround(-1.351, 1)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+bround(-1.25, 1),
+bround(-1.35, 1),
+bround(-1.249, 1),
+bround(-1.349, 1),
+bround(-1.251, 1),
+bround(-1.351, 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+-1.2	-1.4	-1.2	-1.3	-1.3	-1.4
+PREHOOK: query: select
+bround(55.0, -1),
+bround(45.0, -1),
+bround(54.9, -1),
+bround(44.9, -1),
+bround(55.1, -1),
+bround(45.1, -1)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+bround(55.0, -1),
+bround(45.0, -1),
+bround(54.9, -1),
+bround(44.9, -1),
+bround(55.1, -1),
+bround(45.1, -1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+60.0	40.0	50.0	40.0	60.0	50.0
+PREHOOK: query: select
+bround(-55.0, -1),
+bround(-45.0, -1),
+bround(-54.9, -1),
+bround(-44.9, -1),
+bround(-55.1, -1),
+bround(-45.1, -1)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+bround(-55.0, -1),
+bround(-45.0, -1),
+bround(-54.9, -1),
+bround(-44.9, -1),
+bround(-55.1, -1),
+bround(-45.1, -1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+-60.0	-40.0	-50.0	-40.0	-60.0	-50.0

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/ql/src/test/results/clientpositive/vector_bround.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_bround.q.out b/ql/src/test/results/clientpositive/vector_bround.q.out
new file mode 100644
index 0000000..85049a8
--- /dev/null
+++ b/ql/src/test/results/clientpositive/vector_bround.q.out
@@ -0,0 +1,86 @@
+PREHOOK: query: create table test_vector_bround(v0 double, v1 double) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_vector_bround
+POSTHOOK: query: create table test_vector_bround(v0 double, v1 double) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_vector_bround
+PREHOOK: query: insert into table test_vector_bround
+values
+(2.5, 1.25),
+(3.5, 1.35),
+(-2.5, -1.25),
+(-3.5, -1.35),
+(2.49, 1.249),
+(3.49, 1.349),
+(2.51, 1.251),
+(3.51, 1.351)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@test_vector_bround
+POSTHOOK: query: insert into table test_vector_bround
+values
+(2.5, 1.25),
+(3.5, 1.35),
+(-2.5, -1.25),
+(-3.5, -1.35),
+(2.49, 1.249),
+(3.49, 1.349),
+(2.51, 1.251),
+(3.51, 1.351)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@test_vector_bround
+POSTHOOK: Lineage: test_vector_bround.v0 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: test_vector_bround.v1 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: explain select bround(v0), bround(v1, 1) from test_vector_bround
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select bround(v0), bround(v1, 1) from test_vector_bround
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: test_vector_bround
+            Statistics: Num rows: 8 Data size: 128 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: bround(v0) (type: double), bround(v1, 1) (type: double)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 8 Data size: 128 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 8 Data size: 128 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Execution mode: vectorized
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select bround(v0), bround(v1, 1) from test_vector_bround
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_vector_bround
+#### A masked pattern was here ####
+POSTHOOK: query: select bround(v0), bround(v1, 1) from test_vector_bround
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test_vector_bround
+#### A masked pattern was here ####
+2.0	1.2
+4.0	1.4
+-2.0	-1.2
+-4.0	-1.4
+2.0	1.2
+3.0	1.3
+3.0	1.3
+4.0	1.4

http://git-wip-us.apache.org/repos/asf/hive/blob/bd90fc34/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
index 7d7fb28..12a3936 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
@@ -51,6 +51,7 @@ public class HiveDecimal implements Comparable<HiveDecimal> {
   public static final int ROUND_FLOOR = BigDecimal.ROUND_FLOOR;
   public static final int ROUND_CEILING = BigDecimal.ROUND_CEILING;
   public static final int ROUND_HALF_UP = BigDecimal.ROUND_HALF_UP;
+  public static final int ROUND_HALF_EVEN = BigDecimal.ROUND_HALF_EVEN;
 
   private BigDecimal bd = BigDecimal.ZERO;