ACCUMULO-3274 Avoid repeatedly toString()'ing a Path in compaction path

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

Branch: refs/heads/1.7
Commit: b6504bb9da4128069b8b9b61a13cbd95041a3494
Parents: d4dbdd5
Author: Josh Elser <els...@apache.org>
Authored: Wed Dec 30 22:08:23 2015 -0500
Committer: Josh Elser <els...@apache.org>
Committed: Wed Dec 30 22:24:24 2015 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/tserver/Compactor.java   | 11 +++++++----
 .../apache/accumulo/tserver/MinorCompactor.java  | 19 ++++++++++---------
 2 files changed, 17 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/b6504bb9/server/tserver/src/main/java/org/apache/accumulo/tserver/Compactor.java
----------------------------------------------------------------------
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/Compactor.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/Compactor.java
index 381f75c..e16240c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Compactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Compactor.java
@@ -68,6 +68,7 @@ import 
org.apache.accumulo.tserver.Tablet.MinorCompactionReason;
 import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
 public class Compactor implements Callable<CompactionStats> {
@@ -331,14 +332,16 @@ public class Compactor implements 
Callable<CompactionStats> {
 
     clearStats();
 
+    final Path outputFilePath = outputFile.path();
+    final String outputFilePathName = outputFilePath.toString();
     String oldThreadName = Thread.currentThread().getName();
     String newThreadName = "MajC compacting " + extent.toString() + " started 
" + dateFormatter.format(new Date()) + " file: " + outputFile;
     Thread.currentThread().setName(newThreadName);
     thread = Thread.currentThread();
     try {
       FileOperations fileFactory = FileOperations.getInstance();
-      FileSystem ns = 
this.fs.getVolumeByPath(outputFile.path()).getFileSystem();
-      mfw = fileFactory.openWriter(outputFile.path().toString(), ns, 
ns.getConf(), acuTableConf);
+      FileSystem ns = this.fs.getVolumeByPath(outputFilePath).getFileSystem();
+      mfw = fileFactory.openWriter(outputFilePathName, ns, ns.getConf(), 
acuTableConf);
 
       Map<String,Set<ByteSequence>> lGroups;
       try {
@@ -370,7 +373,7 @@ public class Compactor implements Callable<CompactionStats> 
{
 
       // Verify the file, since hadoop 0.20.2 sometimes lies about the success 
of close()
       try {
-        FileSKVIterator openReader = 
fileFactory.openReader(outputFile.path().toString(), false, ns, ns.getConf(), 
acuTableConf);
+        FileSKVIterator openReader = 
fileFactory.openReader(outputFilePathName, false, ns, ns.getConf(), 
acuTableConf);
         openReader.close();
       } catch (IOException ex) {
         log.error("Verification of successful compaction fails!!! " + extent + 
" " + outputFile, ex);
@@ -380,7 +383,7 @@ public class Compactor implements Callable<CompactionStats> 
{
       log.debug(String.format("Compaction %s %,d read | %,d written | %,6d 
entries/sec | %6.3f secs", extent, majCStats.getEntriesRead(),
           majCStats.getEntriesWritten(), (int) (majCStats.getEntriesRead() / 
((t2 - t1) / 1000.0)), (t2 - t1) / 1000.0));
 
-      
majCStats.setFileSize(fileFactory.getFileSize(outputFile.path().toString(), ns, 
ns.getConf(), acuTableConf));
+      majCStats.setFileSize(fileFactory.getFileSize(outputFilePathName, ns, 
ns.getConf(), acuTableConf));
       return majCStats;
     } catch (IOException e) {
       log.error(e, e);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b6504bb9/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactor.java
----------------------------------------------------------------------
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactor.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactor.java
index ba1185a..eadca6d 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactor.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactor.java
@@ -81,7 +81,8 @@ public class MinorCompactor extends Compactor {
 
   @Override
   public CompactionStats call() {
-    log.debug("Begin minor compaction " + getOutputFile() + " " + getExtent());
+    final String outputFileName = getOutputFile();
+    log.debug("Begin minor compaction " + outputFileName + " " + getExtent());
 
     // output to new MapFile with a temporary name
     int sleepTime = 100;
@@ -99,19 +100,19 @@ public class MinorCompactor extends Compactor {
           // (int)(map.size()/((t2 - t1)/1000.0)), (t2 - t1)/1000.0, 
estimatedSizeInBytes()));
 
           if (reportedProblem) {
-            
ProblemReports.getInstance().deleteProblemReport(getExtent().getTableId().toString(),
 ProblemType.FILE_WRITE, getOutputFile());
+            
ProblemReports.getInstance().deleteProblemReport(getExtent().getTableId().toString(),
 ProblemType.FILE_WRITE, outputFileName);
           }
 
           return ret;
         } catch (IOException e) {
-          log.warn("MinC failed (" + e.getMessage() + ") to create " + 
getOutputFile() + " retrying ...");
-          ProblemReports.getInstance().report(new 
ProblemReport(getExtent().getTableId().toString(), ProblemType.FILE_WRITE, 
getOutputFile(), e));
+          log.warn("MinC failed (" + e.getMessage() + ") to create " + 
outputFileName + " retrying ...");
+          ProblemReports.getInstance().report(new 
ProblemReport(getExtent().getTableId().toString(), ProblemType.FILE_WRITE, 
outputFileName, e));
           reportedProblem = true;
         } catch (RuntimeException e) {
           // if this is coming from a user iterator, it is possible that the 
user could change the iterator config and that the
           // minor compaction would succeed
-          log.warn("MinC failed (" + e.getMessage() + ") to create " + 
getOutputFile() + " retrying ...", e);
-          ProblemReports.getInstance().report(new 
ProblemReport(getExtent().getTableId().toString(), ProblemType.FILE_WRITE, 
getOutputFile(), e));
+          log.warn("MinC failed (" + e.getMessage() + ") to create " + 
outputFileName + " retrying ...", e);
+          ProblemReports.getInstance().report(new 
ProblemReport(getExtent().getTableId().toString(), ProblemType.FILE_WRITE, 
outputFileName, e));
           reportedProblem = true;
         } catch (CompactionCanceledException e) {
           throw new IllegalStateException(e);
@@ -126,11 +127,11 @@ public class MinorCompactor extends Compactor {
 
         // clean up
         try {
-          if (getFileSystem().exists(new Path(getOutputFile()))) {
-            getFileSystem().deleteRecursively(new Path(getOutputFile()));
+          if (getFileSystem().exists(new Path(outputFileName))) {
+            getFileSystem().deleteRecursively(new Path(outputFileName));
           }
         } catch (IOException e) {
-          log.warn("Failed to delete failed MinC file " + getOutputFile() + " 
" + e.getMessage());
+          log.warn("Failed to delete failed MinC file " + outputFileName + " " 
+ e.getMessage());
         }
 
         if (isTableDeleting())

Reply via email to