Author: olga
Date: Wed Oct 28 20:51:10 2009
New Revision: 830757
URL: http://svn.apache.org/viewvc?rev=830757&view=rev
Log:
PIG-1052: FINDBUGS: remaining performance warningse(olgan)
Modified:
hadoop/pig/trunk/CHANGES.txt
hadoop/pig/trunk/src/org/apache/pig/PigServer.java
hadoop/pig/trunk/src/org/apache/pig/StandAloneParser.java
hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/Launcher.java
hadoop/pig/trunk/src/org/apache/pig/builtin/PigStorage.java
hadoop/pig/trunk/src/org/apache/pig/data/DataType.java
hadoop/pig/trunk/src/org/apache/pig/impl/io/BufferedPositionedInputStream.java
hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOCross.java
hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOForEach.java
hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java
hadoop/pig/trunk/src/org/apache/pig/impl/plan/CompilationMessageCollector.java
hadoop/pig/trunk/src/org/apache/pig/impl/plan/PlanPrinter.java
hadoop/pig/trunk/src/org/apache/pig/impl/plan/optimizer/RulePlanPrinter.java
hadoop/pig/trunk/src/org/apache/pig/tools/parameters/PreprocessorContext.java
hadoop/pig/trunk/test/findbugsExcludeFile.xml
Modified: hadoop/pig/trunk/CHANGES.txt
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
--- hadoop/pig/trunk/CHANGES.txt (original)
+++ hadoop/pig/trunk/CHANGES.txt Wed Oct 28 20:51:10 2009
@@ -26,6 +26,8 @@
IMPROVEMENTS
+PIG-1052: FINDBUGS: remaining performance warningse(olgan)
+
PIG-1037: Converted sorted and distinct bags to use the new active spilling
paradigm (yinghe via gates)
Modified: hadoop/pig/trunk/src/org/apache/pig/PigServer.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/PigServer.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/PigServer.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/PigServer.java Wed Oct 28 20:51:10 2009
@@ -320,14 +320,17 @@
}
if (urls.hasMoreElements()) {
- String logMessage = "Found multiple resources that match "
- + jarName + ": " + resourceLocation;
+ StringBuffer sb = new StringBuffer("Found multiple resources that
match ");
+ sb.append(jarName);
+ sb.append(": ");
+ sb.append(resourceLocation);
while (urls.hasMoreElements()) {
- logMessage += (logMessage + urls.nextElement() + "; ");
+ sb.append(urls.nextElement());
+ sb.append("; ");
}
- log.debug(logMessage);
+ log.debug(sb.toString());
}
return resourceLocation;
Modified: hadoop/pig/trunk/src/org/apache/pig/StandAloneParser.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/StandAloneParser.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/StandAloneParser.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/StandAloneParser.java Wed Oct 28
20:51:10 2009
@@ -72,10 +72,10 @@
pig.registerQuery(query);
System.out.print("Current aliases: ");
Map<String, LogicalPlan> aliasPlan = pig.getAliases();
- for (Iterator<String> it = aliasPlan.keySet().iterator();
it.hasNext(); ) {
- String alias = it.next();
- LogicalPlan lp = aliasPlan.get(alias);
- System.out.print(alias + "->" +
lp.getLeaves().get(0).getSchema());
+ for (Iterator<Map.Entry<String,LogicalPlan>> it =
aliasPlan.entrySet().iterator(); it.hasNext(); ) {
+ Map.Entry<String, LogicalPlan> e = it.next();
+ LogicalPlan lp = e.getValue();
+ System.out.print(e.getKey() + "->" +
lp.getLeaves().get(0).getSchema());
if (it.hasNext()) System.out.print(", \n");
else System.out.print("\n");
}
Modified:
hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/Launcher.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/Launcher.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
---
hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/Launcher.java
(original)
+++
hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/Launcher.java
Wed Oct 28 20:51:10 2009
@@ -60,7 +60,7 @@
String newLine = "\n";
boolean pigException = false;
boolean outOfMemory = false;
- final String OOM_ERR = "OutOfMemoryError";
+ static final String OOM_ERR = "OutOfMemoryError";
protected List<FileSpec> succeededStores = null;
protected List<FileSpec> failedStores = null;
@@ -574,12 +574,15 @@
String declaringClass = items[0];
//the last member is always the method name
String methodName = items[items.length - 1];
+ StringBuilder sb = new StringBuilder();
//concatenate the names by adding the dot (.) between the members till
the penultimate member
for(int i = 1; i < items.length - 1; ++i) {
- declaringClass += ".";
- declaringClass += items[i];
+ sb.append('.');
+ sb.append(items[i]);
}
+
+ declaringClass = sb.toString();
//from the file details extract the file name and the line number
//PigMapOnly.java:65
Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/PigStorage.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/PigStorage.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/PigStorage.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/PigStorage.java Wed Oct 28
20:51:10 2009
@@ -214,15 +214,15 @@
boolean mapHasNext = false;
Map<String, Object> m = (Map<String, Object>)field;
mOut.write(mapBeginDelim.getBytes(UTF8));
- for(String s: m.keySet()) {
+ for(Map.Entry<String, Object> e: m.entrySet()) {
if(mapHasNext) {
mOut.write(fieldDelim.getBytes(UTF8));
} else {
mapHasNext = true;
}
- putField(s);
+ putField(e.getKey());
mOut.write(mapKeyValueDelim.getBytes(UTF8));
- putField(m.get(s));
+ putField(e.getValue());
}
mOut.write(mapEndDelim.getBytes(UTF8));
break;
Modified: hadoop/pig/trunk/src/org/apache/pig/data/DataType.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DataType.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/data/DataType.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/data/DataType.java Wed Oct 28 20:51:10
2009
@@ -24,6 +24,7 @@
import java.util.Iterator;
import java.util.Map;
import java.util.TreeMap;
+import java.util.Set;
import org.apache.pig.PigException;
import org.apache.pig.backend.executionengine.ExecException;
@@ -912,15 +913,15 @@
boolean hasNext = false;
StringBuilder sb = new StringBuilder();
sb.append("[");
- for(String s: m.keySet()) {
+ for(Map.Entry<String, Object> e: m.entrySet()) {
if(hasNext) {
sb.append(",");
} else {
hasNext = true;
}
- sb.append(s);
+ sb.append(e.getKey());
sb.append("#");
- Object val = m.get(s);
+ Object val = e.getValue();
if(val != null) {
sb.append(val.toString());
}
Modified:
hadoop/pig/trunk/src/org/apache/pig/impl/io/BufferedPositionedInputStream.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/io/BufferedPositionedInputStream.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
---
hadoop/pig/trunk/src/org/apache/pig/impl/io/BufferedPositionedInputStream.java
(original)
+++
hadoop/pig/trunk/src/org/apache/pig/impl/io/BufferedPositionedInputStream.java
Wed Oct 28 20:51:10 2009
@@ -33,7 +33,7 @@
public class BufferedPositionedInputStream extends InputStream {
long pos;
InputStream in;
- final int bufSize = 1024;
+ static final int bufSize = 1024;
public BufferedPositionedInputStream(InputStream in, long pos) {
// Don't buffer a bzip stream as it will cause problems for
split
Modified: hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOCross.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOCross.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOCross.java
(original)
+++ hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOCross.java Wed Oct
28 20:51:10 2009
@@ -123,29 +123,33 @@
log.debug(" fss.size: " + fss.size());
boolean duplicates = false;
Set<String> duplicateAliases = new HashSet<String>();
- for(String alias: aliases.keySet()) {
- Integer count = aliases.get(alias);
+ for(Map.Entry<String, Integer> e: aliases.entrySet()) {
+ Integer count = e.getValue();
if(count > 1) {
Boolean inFlatten = false;
log.debug("inFlatten: " + inFlatten + "
inverseFlattenAlias: " + inverseFlattenAlias);
- inFlatten = inverseFlattenAlias.get(alias);
+ inFlatten = inverseFlattenAlias.get(e.getKey());
log.debug("inFlatten: " + inFlatten + "
inverseFlattenAlias: " + inverseFlattenAlias);
if((null != inFlatten) && (!inFlatten)) {
duplicates = true;
- duplicateAliases.add(alias);
+ duplicateAliases.add(e.getKey());
}
}
}
if(duplicates) {
- String errMessage = "Found duplicates in schema. ";
+ String errMessage = null;
+ StringBuilder sb = new StringBuilder("Found duplicates in
schema. ");
if(duplicateAliases.size() > 0) {
Iterator<String> iter = duplicateAliases.iterator();
- errMessage += ": " + iter.next();
+ sb.append(": ");
+ sb.append(iter.next());
while(iter.hasNext()) {
- errMessage += ", " + iter.next();
+ sb.append(", ");
+ sb.append(iter.next());
}
}
- errMessage += ". Please alias the columns with unique names.";
+ sb.append(". Please alias the columns with unique names.");
+ errMessage = sb.toString();
int errCode = 1007;
throw new FrontendException(errMessage, errCode,
PigException.INPUT, false, null);
}
Modified: hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOForEach.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOForEach.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOForEach.java
(original)
+++ hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOForEach.java Wed
Oct 28 20:51:10 2009
@@ -371,32 +371,40 @@
log.debug(" fss.size: " + fss.size());
boolean duplicates = false;
Map<String, Integer> duplicateAliases = new
HashMap<String, Integer>();
- for(String alias: aliases.keySet()) {
- Integer count = aliases.get(alias);
+ for(Map.Entry<String, Integer> e: aliases.entrySet()) {
+ Integer count = e.getValue();
if(count > 1) {//not checking for null here as
counts are intitalized to 1
Boolean inFlatten = false;
log.debug("inFlatten: " + inFlatten + "
inverseFlattenAlias: " + inverseFlattenAlias);
- inFlatten =
inverseFlattenAlias.get(alias);
+ inFlatten =
inverseFlattenAlias.get(e.getKey());
log.debug("inFlatten: " + inFlatten + "
inverseFlattenAlias: " + inverseFlattenAlias);
if((null == inFlatten) || (!inFlatten))
{
duplicates = true;
- duplicateAliases.put(alias,
count);
+
duplicateAliases.put(e.getKey(), count);
}
}
}
if(duplicates) {
- String errMessage = "Found duplicates in
schema. ";
+ String errMessage;
+ StringBuffer sb = new StringBuffer("Found duplicates in
schema. ");
if(duplicateAliases.size() > 0) {
- Set<String> duplicateCols =
duplicateAliases.keySet();
- Iterator<String> iter =
duplicateCols.iterator();
- String col = iter.next();
- errMessage += col + ": " +
duplicateAliases.get(col) + " columns";
+ Set<Map.Entry<String, Integer>> es =
duplicateAliases.entrySet();
+ Iterator<Map.Entry<String, Integer>>
iter = es.iterator();
+ Map.Entry<String, Integer> e =
iter.next();
+ sb.append(": ");
+ sb.append(e.getValue());
+ sb.append(" columns");
while(iter.hasNext()) {
- col = iter.next();
- errMessage += ", " + col + ": "
+ duplicateAliases.get(col) + " columns";
+ e = iter.next();
+ sb.append(", ");
+ sb.append(e.getKey());
+ sb.append(": ");
+ sb.append(e.getValue());
+ sb.append(" columns");
}
}
- errMessage += ". Please alias the columns with
unique names.";
+ sb.append(". Please alias the columns with unique names.");
+ errMessage = sb.toString();
log.debug(errMessage);
int errCode = 1007;
throw new FrontendException(errMessage,
errCode, PigException.INPUT, false, null);
Modified:
hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
---
hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java
(original)
+++
hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java
Wed Oct 28 20:51:10 2009
@@ -3186,9 +3186,9 @@
MultiMap<String, FuncSpec> loadFuncSpecMap = new MultiMap<String,
FuncSpec>();
if(canonicalMap.keySet().size() > 0) {
- for(String canonicalName: canonicalMap.keySet()) {
- if((null == parentCanonicalName) ||
(parentCanonicalName.equals(canonicalName))) {
- FuncSpec lfSpec =
getLoadFuncSpec(canonicalMap.get(canonicalName), parentCanonicalName);
+ for(Map.Entry<String, LogicalOperator> e: canonicalMap.entrySet())
{
+ if((null == parentCanonicalName) ||
(parentCanonicalName.equals(e.getKey()))) {
+ FuncSpec lfSpec = getLoadFuncSpec(e.getValue(),
parentCanonicalName);
if(null != lfSpec)
loadFuncSpecMap.put(lfSpec.getClassName(), lfSpec);
}
}
Modified:
hadoop/pig/trunk/src/org/apache/pig/impl/plan/CompilationMessageCollector.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/plan/CompilationMessageCollector.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
---
hadoop/pig/trunk/src/org/apache/pig/impl/plan/CompilationMessageCollector.java
(original)
+++
hadoop/pig/trunk/src/org/apache/pig/impl/plan/CompilationMessageCollector.java
Wed Oct 28 20:51:10 2009
@@ -136,12 +136,12 @@
if (nullCounterCount!=0 && aggMap.size()>1) //
PigWarning.NULL_COUNTER_COUNT is definitely in appMap
logMessage("Unable to retrieve hadoop counter for " +
nullCounterCount +
" jobs, the number following warnings may not be correct",
messageType, log);
- for(Enum e: aggMap.keySet()) {
- if (e!=PigWarning.NULL_COUNTER_COUNT)
+ for(Map.Entry<Enum, Long> e: aggMap.entrySet()) {
+ if (e.getKey() !=PigWarning.NULL_COUNTER_COUNT)
{
- Long count = aggMap.get(e);
+ Long count = e.getValue();
if(count != null && count > 0) {
- String message = "Encountered " + messageType + " " +
e.toString() + " " + count + " time(s).";
+ String message = "Encountered " + messageType + " " +
e.getKey().toString() + " " + count + " time(s).";
logMessage(message, messageType, log);
}
}
Modified: hadoop/pig/trunk/src/org/apache/pig/impl/plan/PlanPrinter.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/plan/PlanPrinter.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/impl/plan/PlanPrinter.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/impl/plan/PlanPrinter.java Wed Oct 28
20:51:10 2009
@@ -43,7 +43,6 @@
private String TAB1 = " ";
private String TABMore = "| ";
private String LSep = "|\n|---";
- private String USep = "| |\n| ";
private int levelCntr = -1;
/**
@@ -84,28 +83,6 @@
return sb.toString();
}
- private String planString(P plan) throws VisitorException, IOException {
- StringBuilder sb = new StringBuilder();
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- if(plan!=null)
- plan.explain(baos, mStream);
- else
- return "";
- sb.append(USep);
- sb.append(shiftStringByTabs(baos.toString(), 2));
- return sb.toString();
- }
-
- private String planString(
- List<P> plans) throws VisitorException, IOException {
- StringBuilder sb = new StringBuilder();
- if(plans!=null)
- for (P plan : plans) {
- sb.append(planString(plan));
- }
- return sb.toString();
- }
-
@SuppressWarnings("unchecked")
private String depthFirst(O node) throws VisitorException, IOException {
StringBuilder sb = new StringBuilder(node.name());
Modified:
hadoop/pig/trunk/src/org/apache/pig/impl/plan/optimizer/RulePlanPrinter.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/plan/optimizer/RulePlanPrinter.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
---
hadoop/pig/trunk/src/org/apache/pig/impl/plan/optimizer/RulePlanPrinter.java
(original)
+++
hadoop/pig/trunk/src/org/apache/pig/impl/plan/optimizer/RulePlanPrinter.java
Wed Oct 28 20:51:10 2009
@@ -43,7 +43,6 @@
private String TAB1 = " ";
private String TABMore = "| ";
private String LSep = "|\n|---";
- private String USep = "| |\n| ";
private int levelCntr = -1;
/**
@@ -83,28 +82,6 @@
return sb.toString();
}
- private String planString(RulePlan rulePlan) throws VisitorException,
IOException {
- StringBuilder sb = new StringBuilder();
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- if(rulePlan!=null)
- rulePlan.explain(baos, mStream);
- else
- return "";
- sb.append(USep);
- sb.append(shiftStringByTabs(baos.toString(), 2));
- return sb.toString();
- }
-
- private String planString(
- List<RulePlan> rulePlans) throws VisitorException, IOException {
- StringBuilder sb = new StringBuilder();
- if(rulePlans!=null)
- for (RulePlan rulePlan : rulePlans) {
- sb.append(planString(rulePlan));
- }
- return sb.toString();
- }
-
private String depthFirst(RuleOperator node) throws VisitorException,
IOException {
StringBuilder sb = new StringBuilder(node.name());
sb.append("\n");
Modified:
hadoop/pig/trunk/src/org/apache/pig/tools/parameters/PreprocessorContext.java
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/tools/parameters/PreprocessorContext.java?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
---
hadoop/pig/trunk/src/org/apache/pig/tools/parameters/PreprocessorContext.java
(original)
+++
hadoop/pig/trunk/src/org/apache/pig/tools/parameters/PreprocessorContext.java
Wed Oct 28 20:51:10 2009
@@ -176,9 +176,12 @@
InputStreamReader isr = new InputStreamReader(p.getInputStream());
br = new BufferedReader(isr);
String line=null;
+ StringBuilder sb = new StringBuilder();
while ( (line = br.readLine()) != null){
- streamData+=(line+"\n");
+ sb.append(line);
+ sb.append("\n");
}
+ streamData = sb.toString();
} catch (IOException e){
RuntimeException rte = new RuntimeException("IO Exception while
executing shell command : "+e.getMessage() , e);
throw rte;
@@ -190,9 +193,12 @@
InputStreamReader isr = new InputStreamReader(p.getErrorStream());
br = new BufferedReader(isr);
String line=null;
+ StringBuilder sb = new StringBuilder();
while ( (line = br.readLine()) != null ) {
- streamError += (line+"\n");
+ sb.append(line);
+ sb.append("\n");
}
+ streamError = sb.toString();
log.debug("Error stream while executing shell command : " +
streamError);
} catch (Exception e) {
RuntimeException rte = new RuntimeException("IO Exception while
executing shell command : "+e.getMessage() , e);
Modified: hadoop/pig/trunk/test/findbugsExcludeFile.xml
URL:
http://svn.apache.org/viewvc/hadoop/pig/trunk/test/findbugsExcludeFile.xml?rev=830757&r1=830756&r2=830757&view=diff
==============================================================================
--- hadoop/pig/trunk/test/findbugsExcludeFile.xml (original)
+++ hadoop/pig/trunk/test/findbugsExcludeFile.xml Wed Oct 28 20:51:10 2009
@@ -57,7 +57,7 @@
<Class
name="org.apache.pig.tools.parameters.PigFileParserTokenManager" />
</Match>
<Match>
- <Class name="org.apache.pig.tools.parameters.ParserException" />
+ <Class name="org.apache.pig.tools.parameters.ParseException" />
</Match>
<!-- Exclude CN_IDIOM_NO_SUPER_CALL issue in logical and physical plans
for now-->