You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by da...@apache.org on 2015/09/17 04:43:34 UTC

svn commit: r1703484 - in /pig/trunk: ./ contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/ contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/

Author: daijy
Date: Thu Sep 17 02:43:31 2015
New Revision: 1703484

URL: http://svn.apache.org/r1703484
Log:
PIG-4673: Built In UDF - REPLACE_MULTI : For a given string, search and replace all occurrences of search keys with replacement values

Added:
    pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/REPLACE_MULTI.java
    pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestBuiltinReplaceMulti.java
Modified:
    pig/trunk/CHANGES.txt

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1703484&r1=1703483&r2=1703484&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Thu Sep 17 02:43:31 2015
@@ -24,6 +24,9 @@ INCOMPATIBLE CHANGES
 
 IMPROVEMENTS
 
+PIG-4673: Built In UDF - REPLACE_MULTI : For a given string, search and replace all occurrences
+ of search keys with replacement values (murali.k.h.rao@gmail.com via daijy)
+
 PIG-4674: TOMAP should infer schema (daijy)
 
 PIG-4676: Upgrade Hive to 1.2.1 (daijy)

Added: pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/REPLACE_MULTI.java
URL: http://svn.apache.org/viewvc/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/REPLACE_MULTI.java?rev=1703484&view=auto
==============================================================================
--- pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/REPLACE_MULTI.java (added)
+++ pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/REPLACE_MULTI.java Thu Sep 17 02:43:31 2015
@@ -0,0 +1,108 @@
+/*
+ * 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.piggybank.evaluation.string;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.PigWarning;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+/**
+ * 
+ * REPLACE_MULTI implements eval function to replace all occurrences of search
+ * keys with replacement values. Search - Replacement values are specified in
+ * Map Example:<code>
+ *      input_data = LOAD 'input_data' as (name); -- name = 'Hello World!'
+ *      replaced_name = FOREACH input_data GENERATE REPLACE_MULTI ( name, [ ' '#'_', '!'#'', 'e'#'a', 'o'#'oo' ] ); -- replaced_name = Halloo_Woorld
+ *      </code>
+ * 
+ * The first argument is the source string on which REPLACE_MULTI operation is
+ * performed. The second argument is a map having search key - replacement value
+ * pairs.
+ *
+ */
+
+public class REPLACE_MULTI extends EvalFunc<String> {
+
+    /**
+     * Method invoked on every tuple during FOREACH evaluation. If source string
+     * or search replacement map is empty or null, source string is returned.
+     * 
+     * @param input
+     *            tuple; First field value is the source string and second field
+     *            is a map having search - replacement values.
+     * @exception java.io.IOException
+     */
+    @Override
+    public String exec(Tuple input) throws IOException {
+        try {
+
+            if (input == null || input.size() != 2)
+                return null;
+
+            String source = (String) input.get(0);
+
+            if (input.get(1) == null)
+                return source;
+
+            Map<String, String> searchReplacementMap = (Map<String, String>) input
+                    .get(1);
+
+            if (source == null || source.trim().equalsIgnoreCase("")
+                    || searchReplacementMap.isEmpty())
+                return source;
+
+            for (Map.Entry<String, String> entry : searchReplacementMap
+                    .entrySet()) {
+                source = source.replaceAll(entry.getKey(), entry.getValue());
+            }
+
+            return source;
+
+        } catch (Exception e) {
+            warn("Failed to process input; error - " + e.getMessage(),
+                    PigWarning.UDF_WARNING_1);
+            return null;
+        }
+    }
+
+    @Override
+    public Schema outputSchema(Schema input) {
+        return new Schema(new Schema.FieldSchema(null, DataType.CHARARRAY));
+    }
+
+    @Override
+    public List<FuncSpec> getArgToFuncMapping() throws FrontendException {
+        List<FuncSpec> funcList = new ArrayList<FuncSpec>();
+        Schema s = new Schema();
+        s.add(new Schema.FieldSchema(null, DataType.CHARARRAY));
+        s.add(new Schema.FieldSchema(null, DataType.MAP));
+        funcList.add(new FuncSpec(this.getClass().getName(), s));
+        return funcList;
+    }
+
+}
\ No newline at end of file

Added: pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestBuiltinReplaceMulti.java
URL: http://svn.apache.org/viewvc/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestBuiltinReplaceMulti.java?rev=1703484&view=auto
==============================================================================
--- pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestBuiltinReplaceMulti.java (added)
+++ pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestBuiltinReplaceMulti.java Thu Sep 17 02:43:31 2015
@@ -0,0 +1,189 @@
+/*
+ * 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.piggybank.test.evaluation.string;
+
+import java.io.File;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.PigServer;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.pig.piggybank.evaluation.string.REPLACE_MULTI;
+import org.apache.pig.test.Util;
+import org.junit.Test;
+import org.junit.Before;
+
+import static junit.framework.Assert.*;
+
+public class TestBuiltinReplaceMulti {
+
+    private static PigServer pigServer;
+
+    @Before
+    public void setUp() throws Exception {
+        pigServer = new PigServer(Util.getLocalTestMode(), new Properties());
+    }
+
+    @Test
+    public void testNullSourceStringToReplaceMultiUDF() throws Exception {
+        Tuple input;
+        String output;
+
+        String inputStr = null;
+
+        List<Object> list = new LinkedList<Object>();
+        EvalFunc<String> strFunc = new REPLACE_MULTI();
+        list.add(inputStr);
+        Map<String, String> map = new HashMap<String, String>();
+        list.add(map);
+        input = TupleFactory.getInstance().newTuple(list);
+        output = strFunc.exec(input);
+        assertNull(output);
+
+    }
+
+    @Test
+    public void testSearchReplacementMapNullToReplaceMultiUDF()
+            throws Exception {
+        Tuple input;
+        String output;
+
+        String inputStr = "Hello World!";
+        String expected = "Hello World!";
+
+        List<Object> list = new LinkedList<Object>();
+        EvalFunc<String> strFunc = new REPLACE_MULTI();
+        list.add(inputStr);
+        Map<String, String> map = null;
+        list.add(map);
+        input = TupleFactory.getInstance().newTuple(list);
+        output = strFunc.exec(input);
+        assertTrue(output.equals(expected));
+
+    }
+
+    @Test
+    public void testEmptyMapToReplaceMultiUDF() throws Exception {
+        Tuple input;
+        String output;
+
+        String inputStr = "Hello World!";
+        String expected = "Hello World!";
+
+        List<Object> list = new LinkedList<Object>();
+        EvalFunc<String> strFunc = new REPLACE_MULTI();
+        list.add(inputStr);
+        Map<String, String> map = new HashMap<String, String>();
+        list.add(map);
+        input = TupleFactory.getInstance().newTuple(list);
+        output = strFunc.exec(input);
+        assertTrue(output.equals(expected));
+
+    }
+
+    @Test
+    public void testReplaceMultiUDF() throws Exception {
+        Tuple input;
+        String output;
+
+        String inputStr = "Hello World!";
+        String expected = "Halloo_Woorld";
+
+        List<Object> list = new LinkedList<Object>();
+        EvalFunc<String> strFunc = new REPLACE_MULTI();
+        list.add(inputStr);
+        Map<String, String> map = new HashMap<String, String>();
+        map.put(" ", "_");
+        map.put("e", "a");
+        map.put("!", "");
+        map.put("o", "oo");
+        list.add(map);
+        input = TupleFactory.getInstance().newTuple(list);
+        output = strFunc.exec(input);
+        assertTrue(output.equals(expected));
+
+    }
+
+    @Test
+    public void testPigScriptForReplaceMultiUDFNullParameters()
+            throws Exception {
+        String inputStr = "Hello World!";
+
+        File inputFile = Util.createInputFile("tmp", "testReplaceMultiIn.txt",
+                new String[] { inputStr });
+
+        // test typed data
+        pigServer.registerQuery("A = LOAD '"
+                + Util.encodeEscape(inputFile.getAbsolutePath())
+                + "' AS (name: chararray);");
+        pigServer
+                .registerQuery("B = FOREACH A GENERATE REPLACE_MULTI(NULL, NULL);");
+
+        Iterator<Tuple> it = pigServer.openIterator("B");
+        assertTrue(it.hasNext());
+        Tuple t = it.next();
+        assertNull(t.get(0));
+    }
+
+    @Test
+    public void testPigScriptForReplaceMultiUDFNullMap() throws Exception {
+        String inputStr = "Hello World!";
+
+        File inputFile = Util.createInputFile("tmp", "testReplaceMultiIn.txt",
+                new String[] { inputStr });
+
+        // test typed data
+        pigServer.registerQuery("A = LOAD '"
+                + Util.encodeEscape(inputFile.getAbsolutePath())
+                + "' AS (name: chararray);");
+        pigServer
+                .registerQuery("B = FOREACH A GENERATE REPLACE_MULTI(name, NULL);");
+
+        Iterator<Tuple> it = pigServer.openIterator("B");
+        assertTrue(it.hasNext());
+        Tuple t = it.next();
+        assertEquals(inputStr, t.get(0));
+    }
+
+    @Test
+    public void testPigScriptForReplaceMultiUDF() throws Exception {
+        String inputStr = "Hello World!";
+
+        File inputFile = Util.createInputFile("tmp", "testReplaceMultiIn.txt",
+                new String[] { inputStr });
+
+        // test typed data
+        pigServer.registerQuery("A = LOAD '"
+                + Util.encodeEscape(inputFile.getAbsolutePath())
+                + "' AS (name: chararray);");
+        pigServer
+                .registerQuery("B = FOREACH A GENERATE REPLACE_MULTI(name, [' '#'_','e'#'a','o'#'oo','!'#'']);");
+
+        Iterator<Tuple> it = pigServer.openIterator("B");
+        assertTrue(it.hasNext());
+        Tuple t = it.next();
+        assertEquals("Halloo_Woorld", t.get(0));
+    }
+
+}