You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ch...@apache.org on 2014/05/09 18:24:43 UTC

svn commit: r1593572 - in /pig/trunk: CHANGES.txt src/org/apache/pig/builtin/DoubleRoundTo.java src/org/apache/pig/builtin/FloatRoundTo.java src/org/apache/pig/builtin/ROUND_TO.java test/org/apache/pig/test/TestBuiltin.java

Author: cheolsoo
Date: Fri May  9 16:24:42 2014
New Revision: 1593572

URL: http://svn.apache.org/r1593572
Log:
PIG-3926: ROUND_TO function: rounds double/float to fixed number of decimal places (mrflip via cheolsoo)

Added:
    pig/trunk/src/org/apache/pig/builtin/DoubleRoundTo.java
    pig/trunk/src/org/apache/pig/builtin/FloatRoundTo.java
    pig/trunk/src/org/apache/pig/builtin/ROUND_TO.java
Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/test/org/apache/pig/test/TestBuiltin.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1593572&r1=1593571&r2=1593572&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Fri May  9 16:24:42 2014
@@ -32,6 +32,8 @@ PIG-2207: Support custom counters for ag
 
 IMPROVEMENTS
 
+PIG-3926: ROUND_TO function: rounds double/float to fixed number of decimal places (mrflip via cheolsoo)
+
 PIG-3901: Organize the Pig properties file and document all properties (mrflip via cheolsoo)
 
 PIG-3867: Added hadoop home to build classpath for build pig with unit test on windows (Sergey Svinarchuk via gates)

Added: pig/trunk/src/org/apache/pig/builtin/DoubleRoundTo.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/builtin/DoubleRoundTo.java?rev=1593572&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/builtin/DoubleRoundTo.java (added)
+++ pig/trunk/src/org/apache/pig/builtin/DoubleRoundTo.java Fri May  9 16:24:42 2014
@@ -0,0 +1,82 @@
+/*
+ * 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.pig.builtin;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
+
+import java.math.BigDecimal;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+
+/**
+ * ROUND_TO safely rounds a number to a given precision by using an intermediate
+ * BigDecimal. The too-often seen trick of doing (1000.0 * ROUND(x/1000)) is not
+ * only hard to read but also fails to produce numerically accurate results.
+ *
+ * Given a single data atom and precision it Returns a double extending to the
+ * given number of decimal places. ROUND_TO(0.9876543, 3) is 0.988;
+ * ROUND_TO(0.9876543, 0) is 1.0.
+ *
+ */
+public class DoubleRoundTo extends EvalFunc<Double>{
+    /**
+     * java level API
+     * @param input expects a numeric value to round and a number of digits to keep
+     * @return output returns a single numeric value, the number with only those digits retained
+     */
+    @Override
+    public Double exec(Tuple input) throws IOException {
+        if (input == null || input.size() < 2)
+            return null;
+
+        try {
+            Double     num    = (Double)input.get(0);
+            Integer    digits = (Integer)input.get(1);
+            BigDecimal bdnum  = BigDecimal.valueOf(num);
+
+            bdnum = bdnum.setScale(digits, BigDecimal.ROUND_HALF_UP);
+            return bdnum.doubleValue();
+        } catch (Exception e){
+            throw new IOException("Caught exception processing input row ", e);
+        }
+    }
+
+    /* (non-Javadoc)
+     * @see org.apache.pig.EvalFunc#getArgToFuncMapping()
+     */
+    @Override
+    public List<FuncSpec> getArgToFuncMapping() throws FrontendException {
+        List<FuncSpec> funcList = new ArrayList<FuncSpec>();
+
+        Schema s_dbl = new Schema();
+        s_dbl.add(new Schema.FieldSchema(null, DataType.DOUBLE));
+        s_dbl.add(new Schema.FieldSchema(null, DataType.INTEGER));
+
+        funcList.add(new FuncSpec(this.getClass().getName(), s_dbl));
+
+        return funcList;
+    }
+}

Added: pig/trunk/src/org/apache/pig/builtin/FloatRoundTo.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/builtin/FloatRoundTo.java?rev=1593572&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/builtin/FloatRoundTo.java (added)
+++ pig/trunk/src/org/apache/pig/builtin/FloatRoundTo.java Fri May  9 16:24:42 2014
@@ -0,0 +1,82 @@
+/*
+ * 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.pig.builtin;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
+
+import java.math.BigDecimal;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+
+/**
+ * ROUND_TO safely rounds a number to a given precision by using an intermediate
+ * BigDecimal. The too-often seen trick of doing (1000.0 * ROUND(x/1000)) is not
+ * only hard to read but also fails to produce numerically accurate results.
+ *
+ * Given a single data atom and precision it Returns a float extending to the
+ * given number of decimal places. ROUND_TO(0.9876543, 3) is 0.988;
+ * ROUND_TO(0.9876543, 0) is 1.0.
+ *
+ */
+public class FloatRoundTo extends EvalFunc<Float>{
+    /**
+     * java level API
+     * @param input expects a numeric value to round and a number of digits to keep
+     * @return output returns a single numeric value, the number with only those digits retained
+     */
+    @Override
+    public Float exec(Tuple input) throws IOException {
+        if (input == null || input.size() < 2)
+            return null;
+
+        try {
+            Float      num    = (Float)input.get(0);
+            Integer    digits = (Integer)input.get(1);
+            BigDecimal bdnum  = BigDecimal.valueOf(num);
+
+            bdnum = bdnum.setScale(digits, BigDecimal.ROUND_HALF_UP);
+            return bdnum.floatValue();
+        } catch (Exception e){
+            throw new IOException("Caught exception processing input row ", e);
+        }
+	}
+
+    /* (non-Javadoc)
+     * @see org.apache.pig.EvalFunc#getArgToFuncMapping()
+     */
+    @Override
+    public List<FuncSpec> getArgToFuncMapping() throws FrontendException {
+        List<FuncSpec> funcList = new ArrayList<FuncSpec>();
+
+        Schema s_flt = new Schema();
+        s_flt.add(new Schema.FieldSchema(null, DataType.FLOAT));
+        s_flt.add(new Schema.FieldSchema(null, DataType.INTEGER));
+
+        funcList.add(new FuncSpec(this.getClass().getName(), s_flt));
+
+        return funcList;
+    }
+}

Added: pig/trunk/src/org/apache/pig/builtin/ROUND_TO.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/builtin/ROUND_TO.java?rev=1593572&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/builtin/ROUND_TO.java (added)
+++ pig/trunk/src/org/apache/pig/builtin/ROUND_TO.java Fri May  9 16:24:42 2014
@@ -0,0 +1,102 @@
+/*
+ * 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.pig.builtin;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
+
+import java.math.BigDecimal;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+
+/**
+ *
+ * ROUND_TO safely rounds a number to a given precision by using an intermediate
+ * BigDecimal. The too-often seen trick of doing (1000.0 * ROUND(x/1000)) is not
+ * only hard to read but also fails to produce numerically accurate results.
+ *
+ * Given a single data atom and precision it Returns a double extending to the
+ * given number of decimal places. ROUND_TO(0.9876543, 3) is 0.988;
+ * ROUND_TO(0.9876543, 0) is 1.0.
+ *
+ */
+public class ROUND_TO extends EvalFunc<Double>{
+    /**
+     * java level API
+     * @param input expects a numeric value to round and a number of digits to keep
+     * @return output returns a single numeric value, the number with only those digits retained
+     */
+    @Override
+    public Double exec(Tuple input) throws IOException {
+        if (input == null || input.size() < 2)
+            return null;
+
+        try {
+            Double     num    = DataType.toDouble(input.get(0));
+            Integer    digits = DataType.toInteger(input.get(1));
+            BigDecimal bdnum  = BigDecimal.valueOf(num);
+
+            bdnum = bdnum.setScale(digits, BigDecimal.ROUND_HALF_UP);
+            return bdnum.doubleValue();
+        } catch (NumberFormatException nfe){
+            System.err.println("Failed to process input; error - " + nfe.getMessage());
+            return null;
+        } catch (Exception e){
+            throw new IOException("Caught exception processing input row ", e);
+        }
+    }
+
+    @Override
+    public Schema outputSchema(Schema input) {
+        return new Schema(new Schema.FieldSchema(
+                getSchemaName(this.getClass().getName().toLowerCase(), input), DataType.DOUBLE));
+    }
+
+    /* (non-Javadoc)
+     * @see org.apache.pig.EvalFunc#getArgToFuncMapping()
+     */
+    @Override
+    public List<FuncSpec> getArgToFuncMapping() throws FrontendException {
+        List<FuncSpec> funcList = new ArrayList<FuncSpec>();
+
+        Schema s_bty = new Schema();
+        s_bty.add(new Schema.FieldSchema(null, DataType.BYTEARRAY));
+        s_bty.add(new Schema.FieldSchema(null, DataType.BYTEARRAY));
+
+        Schema s_dbl = new Schema();
+        s_dbl.add(new Schema.FieldSchema(null, DataType.DOUBLE));
+        s_dbl.add(new Schema.FieldSchema(null, DataType.INTEGER));
+
+        Schema s_flt = new Schema();
+        s_flt.add(new Schema.FieldSchema(null, DataType.FLOAT));
+        s_flt.add(new Schema.FieldSchema(null, DataType.INTEGER));
+
+        funcList.add(new FuncSpec(this.getClass().getName(),     s_bty));
+        funcList.add(new FuncSpec(DoubleRoundTo.class.getName(), s_dbl));
+        funcList.add(new FuncSpec(FloatRoundTo.class.getName(),  s_flt));
+
+        return funcList;
+    }
+}

Modified: pig/trunk/test/org/apache/pig/test/TestBuiltin.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestBuiltin.java?rev=1593572&r1=1593571&r2=1593572&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestBuiltin.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestBuiltin.java Fri May  9 16:24:42 2014
@@ -83,6 +83,8 @@ import org.apache.pig.builtin.PigStorage
 import org.apache.pig.builtin.REGEX_EXTRACT;
 import org.apache.pig.builtin.REGEX_EXTRACT_ALL;
 import org.apache.pig.builtin.REPLACE;
+import org.apache.pig.builtin.ROUND;
+import org.apache.pig.builtin.ROUND_TO;
 import org.apache.pig.builtin.RTRIM;
 import org.apache.pig.builtin.SIZE;
 import org.apache.pig.builtin.STRSPLIT;
@@ -1701,6 +1703,40 @@ public class TestBuiltin {
     }
 
     @Test
+    public void testROUND() throws Exception {
+        Double         dbl     = 0.987654321d;
+        Float          flt     = 0.987654321f;
+        EvalFunc<Long> rounder = new ROUND();
+        Tuple          tup     = TupleFactory.getInstance().newTuple(1);
+        long           expected, output;
+
+        tup.set(0, dbl);
+        expected = Math.round(dbl);
+        output   = rounder.exec(tup);
+        assertTrue(output == expected);
+
+        tup.set(0, flt);
+        expected = Math.round(flt);
+        output   = rounder.exec(tup);
+        assertTrue(output == expected);
+    }
+
+    @Test
+    public void testROUND_TO() throws Exception {
+        Double           dbl     = 3.1415925000d, dbl_out;
+        EvalFunc<Double> rounder = new ROUND_TO();
+        Tuple            tup     = TupleFactory.getInstance().newTuple(2);
+        String           expected;
+
+        // Returns double given double
+        tup.set(0, dbl);
+        expected = "3.141593"; tup.set(1, 6); dbl_out = rounder.exec(tup);  assertEquals(expected, dbl_out.toString());
+        expected = "3.1416";   tup.set(1, 4); dbl_out = rounder.exec(tup);  assertEquals(expected, dbl_out.toString());
+        expected = "3.1";      tup.set(1, 1); dbl_out = rounder.exec(tup);  assertEquals(expected, dbl_out.toString());
+        expected = "3.0";      tup.set(1, 0); dbl_out = rounder.exec(tup);  assertEquals(expected, dbl_out.toString());
+    }
+
+    @Test
     public void testStringFuncs() throws Exception {
         // Since String functions are trivial we add test on per case basis
         String inputStr = "Hello World!";