Return-Path: Delivered-To: apmail-hadoop-pig-commits-archive@www.apache.org Received: (qmail 53991 invoked from network); 23 Jun 2010 03:38:20 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 23 Jun 2010 03:38:20 -0000 Received: (qmail 16902 invoked by uid 500); 23 Jun 2010 03:38:20 -0000 Delivered-To: apmail-hadoop-pig-commits-archive@hadoop.apache.org Received: (qmail 16841 invoked by uid 500); 23 Jun 2010 03:38:19 -0000 Mailing-List: contact pig-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: pig-dev@hadoop.apache.org Delivered-To: mailing list pig-commits@hadoop.apache.org Received: (qmail 16834 invoked by uid 500); 23 Jun 2010 03:38:19 -0000 Delivered-To: apmail-incubator-pig-commits@incubator.apache.org Received: (qmail 16831 invoked by uid 99); 23 Jun 2010 03:38:19 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 23 Jun 2010 03:38:19 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 23 Jun 2010 03:38:12 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 8CB3123889B6; Wed, 23 Jun 2010 03:37:13 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r957100 [2/2] - in /hadoop/pig/trunk: ./ src/org/apache/pig/builtin/ test/org/apache/pig/test/ Date: Wed, 23 Jun 2010 03:37:13 -0000 To: pig-commits@incubator.apache.org From: daijy@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20100623033721.8CB3123889B6@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Added: hadoop/pig/trunk/src/org/apache/pig/builtin/REGEX_EXTRACT_ALL.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/REGEX_EXTRACT_ALL.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/REGEX_EXTRACT_ALL.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/REGEX_EXTRACT_ALL.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,107 @@ +/* + * 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.ArrayList; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.pig.EvalFunc; +import org.apache.pig.FuncSpec; +import org.apache.pig.data.DataType; +import org.apache.pig.data.Tuple; +import org.apache.pig.data.TupleFactory; +import org.apache.pig.impl.logicalLayer.FrontendException; +import org.apache.pig.impl.logicalLayer.schema.Schema; + +/** + *
+ *
Syntax:
+ *
String RegexExtractAll(String expression, String regex).
+ *
Input:
+ *
expression-source string.
+ *
regex-regular expression.
+ *
Output:
+ *
A tuple of matched strings.
+ *
+ */ + +public class REGEX_EXTRACT_ALL extends EvalFunc { + + private static TupleFactory tupleFactory = TupleFactory.getInstance(); + + @Override + public Tuple exec(Tuple input) throws IOException { + if (input.size()!=2) { + String msg = "RegexExtractAll : Only 2 parameters are allowed."; + throw new IOException(msg); + } + + if (input.get(0)==null) + return null; + try { + if (!input.get(1).equals(mExpression)) { + try { + mExpression = (String)input.get(1); + mPattern = Pattern.compile(mExpression); + } catch (Exception e) { + String msg = "RegexExtractAll : Mal-Formed Regular expression : "+input.get(1); + throw new IOException(msg); + } + } + } catch (NullPointerException e) { + String msg = "RegexExtractAll : Regular expression is null"; + throw new IOException(msg); + } + + Matcher m = mPattern.matcher((String)input.get(0)); + if (!m.matches()) { + return null; + } + Tuple result = tupleFactory.newTuple(m.groupCount()); + for (int i = 0; i< m.groupCount(); i++) { + result.set(i, m.group(i+1)); + } + return result; + } + + String mExpression = null; + Pattern mPattern = null; + @Override + public Schema outputSchema(Schema input) { + try { + return new Schema(new Schema.FieldSchema(getSchemaName(this.getClass().getName().toLowerCase(), input), + DataType.TUPLE)); + } catch (Exception e) { + return null; + } + } + + @Override + public List getArgToFuncMapping() throws FrontendException { + List funcList = new ArrayList(); + Schema s = new Schema(); + s.add(new Schema.FieldSchema(null, DataType.CHARARRAY)); + s.add(new Schema.FieldSchema(null, DataType.CHARARRAY)); + funcList.add(new FuncSpec(this.getClass().getName(), s)); + return funcList; + } +} + Added: hadoop/pig/trunk/src/org/apache/pig/builtin/REPLACE.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/REPLACE.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/REPLACE.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/REPLACE.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,66 @@ +/* + * 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 org.apache.pig.EvalFunc; +import org.apache.pig.data.Tuple; +import org.apache.pig.data.DataType; +import org.apache.pig.impl.logicalLayer.schema.Schema; + + +/** + * REPLACE implements eval function to replace part of a string. + * Example: + * A = load 'mydata' as (name); + * B = foreach A generate REPLACE(name, 'blabla', 'bla'); + * The first argument is a string on which to perform the operation. The second argument + * is treated as a regular expression. The third argument is the replacement string. + * This is a wrapper around Java's String.replaceAll(String, String); + * + */ +public class REPLACE extends EvalFunc +{ + /** + * Method invoked on every tuple during foreach evaluation + * @param input tuple; first column is assumed to have the column to convert + * @exception java.io.IOException + */ + public String exec(Tuple input) throws IOException { + if (input == null || input.size() < 3) + return null; + + try{ + String source = (String)input.get(0); + String target = (String)input.get(1); + String replacewith = (String)input.get(2); + return source.replaceAll(target, replacewith); + }catch(Exception e){ + log.warn("Failed to process input; error - " + e.getMessage()); + return null; + } + } + + @Override + public Schema outputSchema(Schema input) { + return new Schema(new Schema.FieldSchema(null, DataType.CHARARRAY)); + } + +} \ No newline at end of file Added: hadoop/pig/trunk/src/org/apache/pig/builtin/ROUND.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/ROUND.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/ROUND.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/ROUND.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,77 @@ +/* + * 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 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 implements a binding to the Java function + * {@link java.lang.Math#round(double) Math.round(double)}. + * Given a single data atom it Returns the closest long to the argument. + * + */ +public class ROUND extends EvalFunc{ + /** + * java level API + * @param input expects a single numeric value + * @return output returns a single numeric value, + * the closest long to the argument + */ + @Override + public Long exec(Tuple input) throws IOException { + if (input == null || input.size() == 0) + return null; + + try{ + Double d = DataType.toDouble(input.get(0)); + return Math.round(d); + } 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.LONG)); + } + + /* (non-Javadoc) + * @see org.apache.pig.EvalFunc#getArgToFuncMapping() + */ + @Override + public List getArgToFuncMapping() throws FrontendException { + List funcList = new ArrayList(); + funcList.add(new FuncSpec(this.getClass().getName(), new Schema(new Schema.FieldSchema(null, DataType.BYTEARRAY)))); + funcList.add(new FuncSpec(DoubleRound.class.getName(), new Schema(new Schema.FieldSchema(null, DataType.DOUBLE)))); + funcList.add(new FuncSpec(FloatRound.class.getName(), new Schema(new Schema.FieldSchema(null, DataType.FLOAT)))); + return funcList; + } +} Added: hadoop/pig/trunk/src/org/apache/pig/builtin/SIN.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/SIN.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/SIN.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/SIN.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,32 @@ +/* + * 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; + +/** + * SIN implements a binding to the Java function + * {@link java.lang.Math#sin(double) Math.sin(double)}. + * Given a single data atom it Returns the sine of the argument. + * + */ +public class SIN extends DoubleBase{ + Double compute(Double input){ + return Math.sin(input); + + } +} Added: hadoop/pig/trunk/src/org/apache/pig/builtin/SINH.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/SINH.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/SINH.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/SINH.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,31 @@ +/* + * 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; + +/** + * SINH implements a binding to the Java function + * {@link java.lang.Math#sinh(double) Math.sinh(double)}. + * Given a single data atom it Returns the hyperbolic sine of the argument. + * + */ +public class SINH extends DoubleBase{ + Double compute(Double input){ + return Math.sinh(input); + } +} Added: hadoop/pig/trunk/src/org/apache/pig/builtin/SPLIT.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/SPLIT.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/SPLIT.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/SPLIT.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,71 @@ +/* + * 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.Arrays; +import java.util.regex.PatternSyntaxException; + +import org.apache.pig.EvalFunc; +import org.apache.pig.data.Tuple; +import org.apache.pig.data.TupleFactory; + +/** + * Wrapper around Java's String.split
+ * input tuple: first column is assumed to have a string to split;
+ * the optional second column is assumed to have the delimiter or regex to split on;
+ * if not provided, it's assumed to be '\s' (space)
+ * the optional third column may provide a limit to the number of results.
+ * If limit is not provided, 0 is assumed, as per Java's split(). + */ + +public class SPLIT extends EvalFunc { + + private final static TupleFactory tupleFactory = TupleFactory.getInstance(); + + /** + * Wrapper around Java's String.split + * @param input tuple; first column is assumed to have a string to split; + * the optional second column is assumed to have the delimiter or regex to split on;
+ * if not provided, it's assumed to be '\s' (space) + * the optional third column may provide a limit to the number of results.
+ * If limit is not provided, 0 is assumed, as per Java's split(). + * @exception java.io.IOException + */ + public Tuple exec(Tuple input) throws IOException { + if (input == null || input.size() < 1) + return null; + try { + String source = (String) input.get(0); + String delim = (input.size() > 1 ) ? (String) input.get(1) : "\\s"; + int length = (input.size() > 2) ? (Integer) input.get(2) : 0; + if (source == null || delim == null) { + return null; + } + String[] splits = source.split(delim, length); + return tupleFactory.newTuple(Arrays.asList(splits)); + } catch (ClassCastException e) { + log.warn("class cast exception at "+e.getStackTrace()[0]); + } catch (PatternSyntaxException e) { + log.warn(e.getMessage()); + } + // this only happens if the try block did not complete normally + return null; + } +} Added: hadoop/pig/trunk/src/org/apache/pig/builtin/SQRT.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/SQRT.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/SQRT.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/SQRT.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,30 @@ +/* + * 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; +/** + * SQRT implements a binding to the Java function + * {@link java.lang.Math#sqrt(double) Math.sqrt(double)}. + * Given a single data atom it Returns the square root of the argument. + * + */ +public class SQRT extends DoubleBase{ + Double compute(Double input){ + return Math.sqrt(input); + } +} Added: hadoop/pig/trunk/src/org/apache/pig/builtin/SUBSTRING.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/SUBSTRING.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/SUBSTRING.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/SUBSTRING.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,70 @@ +/* + * 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 org.apache.pig.EvalFunc; +import org.apache.pig.data.DataType; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.schema.Schema; + +/** + * SUBSTRING implements eval function to get a part of a string. + * Example: + * A = load 'mydata' as (name); + * B = foreach A generate SUBSTRING(name, 10, 12); + * + * First argument is the string to take a substring of.
+ * Second argument is the index of the first character of substring.
+ * Third argument is the index of the last character of substring.
+ * if the last argument is past the end of the string, substring of (beginIndex, length(str)) is returned. + */ +public class SUBSTRING extends EvalFunc { + + /** + * Method invoked on every tuple during foreach evaluation + * @param input tuple; first column is assumed to have the column to convert + * @exception java.io.IOException + */ + public String exec(Tuple input) throws IOException { + if (input == null || input.size() < 3) { + log.warn("invalid number of arguments to SUBSTRING"); + return null; + } + try { + String source = (String)input.get(0); + Integer beginindex = (Integer)input.get(1); + Integer endindex = (Integer)input.get(2); + return source.substring(beginindex, Math.min(source.length(), endindex)); + } catch (NullPointerException npe) { + log.warn(npe.toString()); + return null; + } catch (ClassCastException e) { + log.warn(e.toString()); + return null; + } + } + + @Override + public Schema outputSchema(Schema input) { + return new Schema(new Schema.FieldSchema(null, DataType.CHARARRAY)); + } + +} \ No newline at end of file Added: hadoop/pig/trunk/src/org/apache/pig/builtin/TAN.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/TAN.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/TAN.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/TAN.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,31 @@ +/* + * 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; + +/** + * TAN implements a binding to the Java function + * {@link java.lang.Math#tan(double) Math.tan(double)}. + * Given a single data atom it Returns the tangent of the argument. + * + */ +public class TAN extends DoubleBase{ + Double compute(Double input){ + return Math.tan(input); + } +} Added: hadoop/pig/trunk/src/org/apache/pig/builtin/TANH.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/TANH.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/TANH.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/TANH.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,32 @@ +/* + * 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; + +/** + * TANH implements a binding to the Java function + * {@link java.lang.Math#tanh(double) Math.tanh(double)}. + * Given a single data atom it Returns the hyperbolic tangent + * of the argument. + * + */ +public class TANH extends DoubleBase{ + Double compute(Double input){ + return Math.tanh(input); + } +} Added: hadoop/pig/trunk/src/org/apache/pig/builtin/TOBAG.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/TOBAG.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/TOBAG.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/TOBAG.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,56 @@ +/* + * 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 org.apache.pig.EvalFunc; +import org.apache.pig.data.BagFactory; +import org.apache.pig.data.DataBag; +import org.apache.pig.data.Tuple; +import org.apache.pig.data.TupleFactory; + +/** + * This class takes a list of items and puts them into a bag + * T = foreach U generate TOBAG($0, $1, $2); + * It's like saying this: + * T = foreach U generate {($0), ($1), ($2)} + */ +public class TOBAG extends EvalFunc { + + @Override + public DataBag exec(Tuple input) throws IOException { + try { + DataBag bag = BagFactory.getInstance().newDefaultBag(); + + for (int i = 0; i < input.size(); ++i) { + final Object object = input.get(i); + if (object != null) { + Tuple tp2 = TupleFactory.getInstance().newTuple(1); + tp2.set(0, object); + bag.add(tp2); + } + } + + return bag; + } catch (Exception ee) { + throw new RuntimeException("Error while creating a bag", ee); + } + } +} Added: hadoop/pig/trunk/src/org/apache/pig/builtin/TOP.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/TOP.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/TOP.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/TOP.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,349 @@ +/* + * 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.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.PriorityQueue; +import java.util.Random; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.pig.Algebraic; +import org.apache.pig.EvalFunc; +import org.apache.pig.FuncSpec; +import org.apache.pig.backend.executionengine.ExecException; +import org.apache.pig.data.BagFactory; +import org.apache.pig.data.DataBag; +import org.apache.pig.data.DataType; +import org.apache.pig.data.Tuple; +import org.apache.pig.data.TupleFactory; +import org.apache.pig.impl.logicalLayer.FrontendException; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema; + +/** + * Top UDF accepts a bag of tuples and returns top-n tuples depending upon the + * tuple field value of type long. Both n and field number needs to be provided + * to the UDF. The UDF iterates through the input bag and just retains top-n + * tuples by storing them in a priority queue of size n+1 where priority is the + * long field. This is efficient as priority queue provides constant time - O(1) + * removal of the least element and O(log n) time for heap restructuring. The + * UDF is especially helpful for turning the nested grouping operation inside + * out and retaining top-n in a nested group. + * + * Assumes all tuples in the bag contain an element of the same type in the compared column. + * + * Sample usage: + * A = LOAD 'test.tsv' as (first: chararray, second: chararray); + * B = GROUP A BY (first, second); + * C = FOREACH B generate FLATTEN(group), COUNT(*) as count; + * D = GROUP C BY first; // again group by first + * topResults = FOREACH D { + * result = Top(10, 2, C); // and retain top 10 occurrences of 'second' in first + * GENERATE FLATTEN(result); + * } + */ +public class TOP extends EvalFunc implements Algebraic{ + private static final Log log = LogFactory.getLog(TOP.class); + static BagFactory mBagFactory = BagFactory.getInstance(); + static TupleFactory mTupleFactory = TupleFactory.getInstance(); + private Random randomizer = new Random(); + + static class TupleComparator implements Comparator { + private final int fieldNum; + private byte datatype; + private boolean typeFound=false; + + public TupleComparator(int fieldNum) { + this.fieldNum = fieldNum; + } + + /* + * (non-Javadoc) + * @see java.util.Comparator#compare(java.lang.Object, java.lang.Object) + */ + @Override + public int compare(Tuple o1, Tuple o2) { + if (o1 == null) + return -1; + if (o2 == null) + return 1; + try { + Object field1 = o1.get(fieldNum); + Object field2 = o2.get(fieldNum); + if (!typeFound) { + datatype = DataType.findType(field1); + typeFound = true; + } + return DataType.compare(field1, field2, datatype, datatype); + } catch (ExecException e) { + throw new RuntimeException("Error while comparing o1:" + o1 + + " and o2:" + o2, e); + } + } + } + + @Override + public DataBag exec(Tuple tuple) throws IOException { + if (tuple == null || tuple.size() < 3) { + return null; + } + try { + int n = (Integer) tuple.get(0); + int fieldNum = (Integer) tuple.get(1); + DataBag inputBag = (DataBag) tuple.get(2); + PriorityQueue store = new PriorityQueue(n + 1, + new TupleComparator(fieldNum)); + updateTop(store, n, inputBag); + DataBag outputBag = mBagFactory.newDefaultBag(); + for (Tuple t : store) { + outputBag.add(t); + } + if (log.isDebugEnabled()) { + if (randomizer.nextInt(1000) == 1) { + log.debug("outputting a bag: "); + for (Tuple t : outputBag) + log.debug("outputting "+t.toDelimitedString("\t")); + log.debug("=================="); + } + } + return outputBag; + } catch (ExecException e) { + throw new RuntimeException("ExecException executing function: ", e); + } catch (Exception e) { + throw new RuntimeException("General Exception executing function: " + e); + } + } + + protected static void updateTop(PriorityQueue store, int limit, DataBag inputBag) { + Iterator itr = inputBag.iterator(); + while (itr.hasNext()) { + Tuple t = itr.next(); + store.add(t); + if (store.size() > limit) + store.poll(); + } + } + + /* + * (non-Javadoc) + * + * @see org.apache.pig.EvalFunc#getArgToFuncMapping() + */ + @Override + public List getArgToFuncMapping() throws FrontendException { + List fields = new ArrayList(3); + fields.add(new Schema.FieldSchema(null, DataType.INTEGER)); + fields.add(new Schema.FieldSchema(null, DataType.INTEGER)); + fields.add(new Schema.FieldSchema(null, DataType.BAG)); + FuncSpec funcSpec = new FuncSpec(this.getClass().getName(), new Schema(fields)); + List funcSpecs = new ArrayList(1); + funcSpecs.add(funcSpec); + return funcSpecs; + } + + @Override + public Schema outputSchema(Schema input) { + try { + if (input.size() < 3) { + return null; + } + Schema.FieldSchema bagFs = new Schema.FieldSchema(null, + input.getField(2).schema, DataType.BAG); + return new Schema(bagFs); + + } catch (Exception e) { + return null; + } + } + + @Override + public String getInitial() { + return Initial.class.getName(); + } + + @Override + public String getIntermed() { + return Intermed.class.getName(); + } + + @Override + public String getFinal() { + return Final.class.getName(); + } + + /* + * Same as normal code-path exec, but outputs a Tuple with the schema + * -- same schema as expected input. + */ + static public class Initial extends EvalFunc { + //private static final Log log = LogFactory.getLog(Initial.class); + //private final Random randomizer = new Random(); + @Override + public Tuple exec(Tuple tuple) throws IOException { + if (tuple == null || tuple.size() < 3) { + return null; + } + + try { + int n = (Integer) tuple.get(0); + int fieldNum = (Integer) tuple.get(1); + DataBag inputBag = (DataBag) tuple.get(2); + Tuple retTuple = mTupleFactory.newTuple(3); + DataBag outputBag = mBagFactory.newDefaultBag(); + // initially, there should only be one, so not much point in doing the priority queue + for (Tuple t : inputBag) { + outputBag.add(t); + } + retTuple.set(0, n); + retTuple.set(1,fieldNum); + retTuple.set(2, outputBag); + return retTuple; + } catch (Exception e) { + throw new RuntimeException("General Exception executing function: " + e); + } + } + } + + static public class Intermed extends EvalFunc { + private static final Log log = LogFactory.getLog(Intermed.class); + private final Random randomizer = new Random(); + /* The input is a tuple that contains a single bag. + * This bag contains outputs of the Initial step -- + * tuples of the format (limit, index, { top_tuples }) + * + * We need to take the top of tops and return a similar tuple. + * + * (non-Javadoc) + * @see org.apache.pig.EvalFunc#exec(org.apache.pig.data.Tuple) + */ + @Override + public Tuple exec(Tuple input) throws IOException { + if (input == null || input.size() < 1) { + return null; + } + try { + DataBag bagOfIntermediates = (DataBag) input.get(0); + Iterator intermediateIterator = bagOfIntermediates.iterator(); + if (!intermediateIterator.hasNext()) { + return null; + } + Tuple peekTuple = intermediateIterator.next(); + if (peekTuple == null || peekTuple.size() < 3 ) return null; + int n = (Integer) peekTuple.get(0); + int fieldNum = (Integer) peekTuple.get(1); + DataBag inputBag = (DataBag) peekTuple.get(2); + + PriorityQueue store = new PriorityQueue(n + 1, + new TupleComparator(fieldNum)); + + updateTop(store, n, inputBag); + + while (intermediateIterator.hasNext()) { + Tuple t = intermediateIterator.next(); + if (t == null || t.size() < 3 ) continue; + updateTop(store, n, (DataBag) t.get(2)); + } + + DataBag outputBag = mBagFactory.newDefaultBag(); + for (Tuple t : store) { + outputBag.add(t); + } + Tuple retTuple = mTupleFactory.newTuple(3); + retTuple.set(0, n); + retTuple.set(1,fieldNum); + retTuple.set(2, outputBag); + if (log.isDebugEnabled()) { + if (randomizer.nextInt(1000) == 1) log.debug("outputting "+retTuple.toDelimitedString("\t")); + } + return retTuple; + } catch (ExecException e) { + throw new RuntimeException("ExecException executing function: ", e); + } catch (Exception e) { + throw new RuntimeException("General Exception executing function: " + e); + } + } + + } + + static public class Final extends EvalFunc { + + private static final Log log = LogFactory.getLog(Final.class); + private final Random randomizer = new Random(); + + + + /* + * The input to this function is a tuple that contains a single bag. + * This bag, in turn, contains outputs of the Intermediate step -- + * tuples of the format (limit, index, { top_tuples } ) + * + * we want to return a bag of top tuples + * + * (non-Javadoc) + * @see org.apache.pig.EvalFunc#exec(org.apache.pig.data.Tuple) + */ + @Override + public DataBag exec(Tuple tuple) throws IOException { + if (tuple == null || tuple.size() < 1) { + return null; + } + try { + DataBag bagOfIntermediates = (DataBag) tuple.get(0); + Iterator intermediateIterator = bagOfIntermediates.iterator(); + if (!intermediateIterator.hasNext()) { + return null; + } + Tuple peekTuple = intermediateIterator.next(); + if (peekTuple == null || peekTuple.size() < 3 ) return null; + int n = (Integer) peekTuple.get(0); + int fieldNum = (Integer) peekTuple.get(1); + DataBag inputBag = (DataBag) peekTuple.get(2); + + PriorityQueue store = new PriorityQueue(n + 1, + new TupleComparator(fieldNum)); + + updateTop(store, n, inputBag); + + while (intermediateIterator.hasNext()) { + Tuple t = intermediateIterator.next(); + if (t == null || t.size() < 3 ) continue; + updateTop(store, n, (DataBag) t.get(2)); + } + + DataBag outputBag = mBagFactory.newDefaultBag(); + for (Tuple t : store) { + outputBag.add(t); + } + if (log.isDebugEnabled()) { + if (randomizer.nextInt(1000) == 1) for (Tuple t : outputBag) log.debug("outputting "+t.toDelimitedString("\t")); + } + return outputBag; + } catch (ExecException e) { + throw new RuntimeException("ExecException executing function: ", e); + } catch (Exception e) { + throw new RuntimeException("General Exception executing function: " + e); + } + } + } +} + Added: hadoop/pig/trunk/src/org/apache/pig/builtin/TOTUPLE.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/TOTUPLE.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/TOTUPLE.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/TOTUPLE.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,65 @@ +/* + * 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.ArrayList; +import java.util.List; + +import org.apache.pig.EvalFunc; +import org.apache.pig.data.DataType; +import org.apache.pig.data.Tuple; +import org.apache.pig.data.TupleFactory; +import org.apache.pig.impl.logicalLayer.schema.Schema; + +/** + * This class makes a tuple out of the parameter + * T = foreach U generate TOTUPLE($0, $1, $2); + * It generates a tuple containing $0, $1, and $2 + */ +public class TOTUPLE extends EvalFunc { + + @Override + public Tuple exec(Tuple input) throws IOException { + try { + List items = new ArrayList(); + for (int i = 0; i < input.size(); ++i) { + items.add(input.get(i)); + } + return TupleFactory.getInstance().newTuple(items); + } catch (Exception e) { + throw new RuntimeException("Error while creating a tuple", e); + } + } + + @Override + public Schema outputSchema(Schema input) { + try { + Schema tupleSchema = new Schema(); + for (int i = 0; i < input.size(); ++i) { + tupleSchema.add(input.getField(i)); + } + return new Schema(new Schema.FieldSchema(getSchemaName(this + .getClass().getName().toLowerCase(), input), tupleSchema, + DataType.TUPLE)); + } catch (Exception e) { + return null; + } + } + +} Added: hadoop/pig/trunk/src/org/apache/pig/builtin/TRIM.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/TRIM.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/TRIM.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/TRIM.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,53 @@ +/* + * 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 org.apache.pig.EvalFunc; +import org.apache.pig.backend.executionengine.ExecException; +import org.apache.pig.data.DataType; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.schema.Schema; + +/** + * Returns a string, with leading and trailing whitespace omitted. + * Implements a binding to the Java function {@link java.lang.String#trim() String.trim()}. + */ +public class TRIM extends EvalFunc { + @Override + public String exec(Tuple input) throws IOException { + if (input == null || input.size() == 0) { + return null; + } + try { + String str = (String) input.get(0); + if (str == null) return null; + if (str.length() == 0) return str; + return str.trim(); + } catch (ExecException e) { + log.warn("Error reading input: " + e.getMessage()); + return null; + } + } + + @Override + public Schema outputSchema(Schema input) { + return new Schema(new Schema.FieldSchema(null, DataType.CHARARRAY)); + } +} Added: hadoop/pig/trunk/src/org/apache/pig/builtin/UCFIRST.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/UCFIRST.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/UCFIRST.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/UCFIRST.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,52 @@ +/* + * 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 org.apache.pig.EvalFunc; +import org.apache.pig.backend.executionengine.ExecException; +import org.apache.pig.data.DataType; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.schema.Schema; + +/** + * upper-case the first character of a string + */ +public class UCFIRST extends EvalFunc { + @Override + public String exec(Tuple input) throws IOException { + if (input == null || input.size() == 0) { + return null; + } + try { + String str = (String) input.get(0); + if (str == null) return null; + if (str.length() == 0) return str; + return Character.toUpperCase(str.charAt(0))+str.substring(1); + } catch (ExecException e) { + log.warn("Error reading input: " + e.getMessage()); + return null; + } + } + + @Override + public Schema outputSchema(Schema input) { + return new Schema(new Schema.FieldSchema(null, DataType.CHARARRAY)); + } +} Added: hadoop/pig/trunk/src/org/apache/pig/builtin/UPPER.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/UPPER.java?rev=957100&view=auto ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/builtin/UPPER.java (added) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/UPPER.java Wed Jun 23 03:37:11 2010 @@ -0,0 +1,89 @@ +/* + * 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 org.apache.pig.EvalFunc; +import org.apache.pig.PigWarning; +import org.apache.pig.data.Tuple; +import org.apache.pig.data.DataType; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.logicalLayer.FrontendException; +import org.apache.pig.FuncSpec; + + +/** + * UPPER implements eval function to convert a string to upper case + * Example: + * A = load 'mydata' as (name); + * B = foreach A generate UPPER(name); + */ +public class UPPER extends EvalFunc { + + /** + * Upper-cases an input string. + * @param input tuple; first column is assumed to have the column to convert + * + * @exception IOException + */ + public String exec(Tuple input) throws IOException { + if (input == null || input.size() == 0 || input.get(0) == null) + return null; + + String str = null; + try { + str = (String)input.get(0); + return str.toUpperCase(); + } + catch (ClassCastException e) { + warn("unable to cast input "+input.get(0)+" of class "+ + input.get(0).getClass()+" to String", PigWarning.UDF_WARNING_1); + return null; + } + catch(Exception e){ + warn("Error processing input "+input.get(0), PigWarning.UDF_WARNING_1); + return null; + } + } + + /** + * This method gives a name to the column. + * @param input - schema of the input data + * @return schema of the input data + */ + @Override + public Schema outputSchema(Schema input) { + return new Schema(new Schema.FieldSchema(getSchemaName(this.getClass().getName().toLowerCase(), input), DataType.CHARARRAY)); + } + + /* (non-Javadoc) + * @see org.apache.pig.EvalFunc#getArgToFuncMapping() + */ + @Override + public List getArgToFuncMapping() throws FrontendException { + List funcList = new ArrayList(); + funcList.add(new FuncSpec(this.getClass().getName(), new Schema(new Schema.FieldSchema(null, DataType.CHARARRAY)))); + + return funcList; + } + +} Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java?rev=957100&r1=957099&r2=957100&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java Wed Jun 23 03:37:11 2010 @@ -17,12 +17,19 @@ */ package org.apache.pig.test; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import java.lang.reflect.Method; import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; import java.util.Map; +import java.util.Random; +import java.util.Set; import org.apache.pig.Algebraic; import org.apache.pig.EvalFunc; @@ -34,18 +41,35 @@ import org.apache.pig.backend.hadoop.dat import org.apache.pig.builtin.ARITY; import org.apache.pig.builtin.BagSize; import org.apache.pig.builtin.CONCAT; +import org.apache.pig.builtin.COR; import org.apache.pig.builtin.COUNT; import org.apache.pig.builtin.COUNT_STAR; +import org.apache.pig.builtin.COV; import org.apache.pig.builtin.DIFF; import org.apache.pig.builtin.Distinct; +import org.apache.pig.builtin.INDEXOF; +import org.apache.pig.builtin.LAST_INDEX_OF; +import org.apache.pig.builtin.LCFIRST; +import org.apache.pig.builtin.LOWER; import org.apache.pig.builtin.MapSize; 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.SIZE; +import org.apache.pig.builtin.SPLIT; +import org.apache.pig.builtin.SUBSTRING; import org.apache.pig.builtin.StringConcat; import org.apache.pig.builtin.StringSize; +import org.apache.pig.builtin.TOBAG; import org.apache.pig.builtin.TOKENIZE; +import org.apache.pig.builtin.TOP; +import org.apache.pig.builtin.TOTUPLE; +import org.apache.pig.builtin.TRIM; import org.apache.pig.builtin.TextLoader; import org.apache.pig.builtin.TupleSize; +import org.apache.pig.builtin.UCFIRST; +import org.apache.pig.builtin.UPPER; import org.apache.pig.data.BagFactory; import org.apache.pig.data.DataBag; import org.apache.pig.data.DataByteArray; @@ -1106,6 +1130,413 @@ public class TestBuiltin { } + @Test + public void testMathFuncs() throws Exception { + Random generator = new Random(); + generator.setSeed(System.currentTimeMillis()); + Double delta = 0.1; + // We assume that UDFs are stored in org.apache.pig.builtin + // Change this test case if we add more hierarchy later\ + // Also, we assume that we have a function with math function + // associated with these UDF with a lowercase name + String[] mathFuncs = { + "SIN", + "SINH", + "ASIN", + "COS", + "COSH", + "ACOS", + "TAN", + "TANH", + "ATAN", + "LOG", + "LOG10", + "SQRT", + "CEIL", + "EXP", + "FLOOR", + "CBRT" + }; + String udfPackage = "org.apache.pig.builtin."; + //String[] mathNonStdFuncs = {}; + EvalFunc evalFunc; + Tuple tup; + Double input, actual, expected; + Method mathMethod; + String msg; + for(String func: mathFuncs) { + evalFunc = (EvalFunc) Class.forName(udfPackage + func).newInstance(); + tup = DefaultTupleFactory.getInstance().newTuple(1); + // double value between 0.0 and 1.0 + input = generator.nextDouble(); + tup.set(0, input); + mathMethod = Math.class.getDeclaredMethod(func.toLowerCase(), double.class); + actual = evalFunc.exec(tup); + expected = (Double)mathMethod.invoke(null, input); + msg = "[Testing " + func + " on input: " + input + " ( (actual) " + actual + " == " + expected + " (expected) )]"; + assertEquals(msg, actual, expected, delta); + } + } + + @Test + public void testStringFuncs() throws Exception { + // Since String functions are trivial we add test on per case basis + String inputStr = "Hello World!"; + String inputStrLower = "hello world!"; + String inputStrUpper = "HELLO WORLD!"; + String inputStrCamel = "hello World!"; + String inputStroWitha = "Hella Warld!"; + String inpuStrExtra = "Hello World! "; + + List l = new LinkedList(); + l.add(inputStr); + l.add("o"); + + String expected = null; + Tuple input; + String output; + Integer intOutput; + EvalFunc strFunc; + EvalFunc intFunc; + + strFunc = new LCFIRST(); + input = DefaultTupleFactory.getInstance().newTuple(inputStr); + expected = inputStrCamel; + output = strFunc.exec(input); + assertTrue(output.equals(expected)); + + strFunc = new UCFIRST(); + input = DefaultTupleFactory.getInstance().newTuple(inputStrCamel); + expected = inputStr; + output = strFunc.exec(input); + assertTrue(output.equals(expected)); + + intFunc = new LAST_INDEX_OF(); + input = DefaultTupleFactory.getInstance().newTuple(l); + intOutput = intFunc.exec(input); + assertTrue(intOutput.intValue()==7); + + intFunc = new INDEXOF(); + input = DefaultTupleFactory.getInstance().newTuple(l); + intOutput = intFunc.exec(input); + assertTrue(intOutput.intValue()==4); + + strFunc = new UPPER(); + input = DefaultTupleFactory.getInstance().newTuple(inputStr); + expected = inputStrUpper; + output = strFunc.exec(input); + assertTrue(output.equals(expected)); + + strFunc = new LOWER(); + input = DefaultTupleFactory.getInstance().newTuple(inputStr); + expected = inputStrLower; + output = strFunc.exec(input); + assertTrue(output.equals(expected)); + + strFunc = new REPLACE(); + l.clear(); + l.add(inputStr); + l.add("o"); + l.add("a"); + input = DefaultTupleFactory.getInstance().newTuple(l); + expected = inputStroWitha; + output = strFunc.exec(input); + assertTrue(output.equals(expected)); + + strFunc = new SUBSTRING(); + l.clear(); + l.add(inputStr); + l.add(1); + l.add(5); + input = DefaultTupleFactory.getInstance().newTuple(l); + expected = "ello"; + output = strFunc.exec(input); + assertTrue(output.equals(expected)); + + strFunc = new TRIM(); + input = DefaultTupleFactory.getInstance().newTuple(inpuStrExtra); + expected = inputStr; + output = strFunc.exec(input); + assertTrue(output.equals(expected)); + + SPLIT splitter = new SPLIT(); + Tuple test1 = TupleFactory.getInstance().newTuple(1); + Tuple test2 = TupleFactory.getInstance().newTuple(2); + Tuple test3 = TupleFactory.getInstance().newTuple(3); + + test2.set(0, "foo"); + test2.set(1, ":"); + Tuple splits = splitter.exec(test2); + assertEquals("no matches should return tuple with original string", 1, splits.size()); + assertEquals("no matches should return tuple with original string", "foo", + splits.get(0)); + + // test default delimiter + test1.set(0, "f ooo bar"); + splits = splitter.exec(test1); + assertEquals("split on default value ", 3, splits.size()); + assertEquals("f", splits.get(0)); + assertEquals("ooo", splits.get(1)); + assertEquals("bar", splits.get(2)); + + // test trimming of whitespace + test1.set(0, "foo bar "); + splits = splitter.exec(test1); + assertEquals("whitespace trimmed if no length arg", 2, splits.size()); + + // test forcing null matches with length param + test3.set(0, "foo bar "); + test3.set(1, "\\s"); + test3.set(2, 10); + splits = splitter.exec(test3); + assertEquals("length forces empty string matches on end", 5, splits.size()); + + // test limiting results with limit + test3.set(0, "foo:bar:baz"); + test3.set(1, ":"); + test3.set(2, 2); + splits = splitter.exec(test3); + assertEquals(2, splits.size()); + assertEquals("foo", splits.get(0)); + assertEquals("bar:baz", splits.get(1)); + + Tuple t1 = TupleFactory.getInstance().newTuple(3); + t1.set(0, "/search/iy/term1/test"); + t1.set(1, "^\\/search\\/iy\\/(.*?)\\/.*"); + t1.set(2, 1); + + Tuple t2 = TupleFactory.getInstance().newTuple(3); + t2.set(0, "/search/iy/term1/test"); + t2.set(1, "^\\/search\\/iy\\/(.*?)\\/.*"); + t2.set(2, 2); + + Tuple t3 = TupleFactory.getInstance().newTuple(3); + t3.set(0, null); + t3.set(1, "^\\/search\\/iy\\/(.*?)\\/.*"); + t3.set(2, 2); + + REGEX_EXTRACT func = new REGEX_EXTRACT(); + String r = func.exec(t1); + assertTrue(r.equals("term1")); + r = func.exec(t2); + assertTrue(r==null); + r = func.exec(t3); + assertTrue(r==null); + + String matchRegex = "^(.+)\\b\\s+is a\\s+\\b(.+)$"; + TupleFactory tupleFactory = TupleFactory.getInstance(); + Tuple te1 = tupleFactory.newTuple(2); + te1.set(0,"this is a match"); + te1.set(1, matchRegex); + + Tuple te2 = tupleFactory.newTuple(2); + te2.set(0, "no match"); + te2.set(1, matchRegex); + + Tuple te3 = tupleFactory.newTuple(2); + te3.set(0, null); + te3.set(1, matchRegex); + + REGEX_EXTRACT_ALL funce = new REGEX_EXTRACT_ALL(); + Tuple re = funce.exec(te1); + assertEquals(re.size(), 2); + assertEquals("this", re.get(0)); + assertEquals("match", re.get(1)); + + re = funce.exec(te2); + assertTrue(re==null); + + re = funce.exec(te3); + assertTrue(re==null); + } + + @Test + public void testStatsFunc() throws Exception { + COV cov = new COV("a","b"); + DataBag dBag = DefaultBagFactory.getInstance().newDefaultBag(); + Tuple tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 1.0); + dBag.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 4.0); + dBag.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 8.0); + dBag.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 4.0); + dBag.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 7.0); + dBag.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 8.0); + dBag.add(tup1); + DataBag dBag1 = DefaultBagFactory.getInstance().newDefaultBag(); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 2.0); + dBag1.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 2.0); + dBag1.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 3.0); + dBag1.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 3.0); + dBag1.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 2.0); + dBag1.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 4.0); + dBag1.add(tup1); + Tuple input = DefaultTupleFactory.getInstance().newTuple(2); + input.set(0, dBag); + input.set(1, dBag1); + DataBag output = cov.exec(input); + Iterator it = output.iterator(); + Tuple ans = (Tuple)it.next(); + assertEquals((String)ans.get(0),"a"); + assertEquals((String)ans.get(1),"b"); + assertEquals(1.11111, (Double)ans.get(2),0.0005); + + COR cor = new COR("a","b"); + dBag = DefaultBagFactory.getInstance().newDefaultBag(); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 1.0); + dBag.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 4.0); + dBag.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 8.0); + dBag.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 4.0); + dBag.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 7.0); + dBag.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 8.0); + dBag.add(tup1); + dBag1 = DefaultBagFactory.getInstance().newDefaultBag(); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 2.0); + dBag1.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 2.0); + dBag1.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 3.0); + dBag1.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 3.0); + dBag1.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 2.0); + dBag1.add(tup1); + tup1 = DefaultTupleFactory.getInstance().newTuple(1); + tup1.set(0, 4.0); + dBag1.add(tup1); + input = DefaultTupleFactory.getInstance().newTuple(2); + input.set(0, dBag); + input.set(1, dBag1); + output = cor.exec(input); + it = output.iterator(); + ans = (Tuple) it.next(); + assertEquals((String)ans.get(0),"a"); + assertEquals((String)ans.get(1),"b"); + assertEquals(0.582222509739582, (Double)ans.get(2) ,0.0005); + } + + private void checkItemsGT(Iterable tuples, int field, int limit) throws ExecException { + for (Tuple t : tuples) { + Long val = (Long) t.get(field); + assertTrue("Value "+ val + " exceeded the expected limit", val > limit); + } + } + + @Test + public void testMiscFunc() throws Exception { + TOBAG tb = new TOBAG(); + + Tuple input = TupleFactory.getInstance().newTuple(); + for (int i = 0; i < 100; ++i) { + input.append(i); + } + + Set s = new HashSet(); + DataBag db = tb.exec(input); + for (Tuple t : db) { + s.add((Integer) t.get(0)); + } + + // finally check the bag had everything we put in the tuple. + assertEquals(100, s.size()); + for (int i = 0; i < 100; ++i) { + assertTrue(s.contains(i)); + } + + TOTUPLE tt = new TOTUPLE(); + + input = TupleFactory.getInstance().newTuple(); + for (int i = 0; i < 100; ++i) { + input.append(i); + } + + Tuple output = tt.exec(input); + assertTrue(!(input == output)); + assertEquals(input, output); + + TOP top = new TOP(); + TupleFactory tupleFactory = DefaultTupleFactory.getInstance(); + BagFactory bagFactory = DefaultBagFactory.getInstance(); + Tuple inputTuple = tupleFactory.newTuple(3); + DataBag dBag = bagFactory.newDefaultBag(); + + // set N = 10 i.e retain top 10 tuples + inputTuple.set(0, 10); + // compare tuples by field number 1 + inputTuple.set(1, 1); + // set the data bag containing the tuples + inputTuple.set(2, dBag); + + // generate tuples of the form (group-1, 1), (group-2, 2) ... + for (long i = 0; i < 100; i++) { + Tuple nestedTuple = tupleFactory.newTuple(2); + nestedTuple.set(0, "group-" + i); + nestedTuple.set(1, i); + dBag.add(nestedTuple); + } + + DataBag outBag = top.exec(inputTuple); + assertEquals(outBag.size(), 10L); + checkItemsGT(outBag, 1, 89); + + // two initial results + Tuple init1 = (new TOP.Initial()).exec(inputTuple); + Tuple init2 = (new TOP.Initial()).exec(inputTuple); + // two intermediate results + + DataBag intermedBag = bagFactory.newDefaultBag(); + intermedBag.add(init1); + intermedBag.add(init2); + Tuple intermedInput = tupleFactory.newTuple(intermedBag); + Tuple intermedOutput1 = (new TOP.Intermed()).exec(intermedInput); + Tuple intermedOutput2 = (new TOP.Intermed()).exec(intermedInput); + checkItemsGT((DataBag)intermedOutput1.get(2), 1, 94); + + // final result + DataBag finalInputBag = bagFactory.newDefaultBag(); + finalInputBag.add(intermedOutput1); + finalInputBag.add(intermedOutput2); + Tuple finalInput = tupleFactory.newTuple(finalInputBag); + outBag = (new TOP.Final()).exec(finalInput); + assertEquals(outBag.size(), 10L); + checkItemsGT(outBag, 1, 96); + } @Test public void testDistinct() throws Exception { @@ -1368,7 +1799,7 @@ public class TestBuiltin { assertTrue(f3.size() == arity3); Util.deleteFile(cluster, "input.txt"); } - + /* @Test public void testLFBin() throws Exception {