You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/02/13 19:49:45 UTC

svn commit: r1445841 - in /hbase/branches/hbase-7290/hbase-server/src: main/java/org/apache/hadoop/hbase/backup/ main/java/org/apache/hadoop/hbase/master/ main/java/org/apache/hadoop/hbase/regionserver/ test/java/org/apache/hadoop/hbase/backup/ test/ja...

Author: jmhsieh
Date: Wed Feb 13 18:49:44 2013
New Revision: 1445841

URL: http://svn.apache.org/r1445841
Log:
HBASE-7423 HFileArchiver should not use the configuration from the Filesystem (Enis Soztutar)


Modified:
    hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
    hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
    hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
    hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
    hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
    hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java?rev=1445841&r1=1445840&r2=1445841&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java Wed Feb 13 18:49:44 2013
@@ -62,20 +62,22 @@ public class HFileArchiver {
   /**
    * Cleans up all the files for a HRegion by archiving the HFiles to the
    * archive directory
+   * @param conf the configuration to use
    * @param fs the file system object
    * @param info HRegionInfo for region to be deleted
    * @throws IOException
    */
-  public static void archiveRegion(FileSystem fs, HRegionInfo info)
+  public static void archiveRegion(Configuration conf, FileSystem fs, HRegionInfo info)
       throws IOException {
-    Path rootDir = FSUtils.getRootDir(fs.getConf());
-    archiveRegion(fs, rootDir, HTableDescriptor.getTableDir(rootDir, info.getTableName()),
+    Path rootDir = FSUtils.getRootDir(conf);
+    archiveRegion(conf, fs, rootDir, HTableDescriptor.getTableDir(rootDir, info.getTableName()),
       HRegion.getRegionDir(rootDir, info));
   }
 
 
   /**
    * Remove an entire region from the table directory via archiving the region's hfiles.
+   * @param conf the configuration to use
    * @param fs {@link FileSystem} from which to remove the region
    * @param rootdir {@link Path} to the root directory where hbase files are stored (for building
    *          the archive path)
@@ -85,7 +87,8 @@ public class HFileArchiver {
    *         operations could not complete.
    * @throws IOException if the request cannot be completed
    */
-  public static boolean archiveRegion(FileSystem fs, Path rootdir, Path tableDir, Path regionDir)
+  public static boolean archiveRegion(Configuration conf, FileSystem fs,
+      Path rootdir, Path tableDir, Path regionDir)
       throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("ARCHIVING region " + regionDir.toString());
@@ -104,7 +107,7 @@ public class HFileArchiver {
 
     // make sure the regiondir lives under the tabledir
     Preconditions.checkArgument(regionDir.toString().startsWith(tableDir.toString()));
-    Path regionArchiveDir = HFileArchiveUtil.getRegionArchiveDir(fs.getConf(), tableDir, regionDir);
+    Path regionArchiveDir = HFileArchiveUtil.getRegionArchiveDir(conf, tableDir, regionDir);
 
     LOG.debug("Have an archive directory, preparing to move files");
     FileStatusConverter getAsFile = new FileStatusConverter(fs);
@@ -180,16 +183,16 @@ public class HFileArchiver {
 
   /**
    * Remove the store files, either by archiving them or outright deletion
+   * @param conf {@link Configuration} to examine to determine the archive directory
    * @param fs the filesystem where the store files live
    * @param parent Parent region hosting the store files
-   * @param conf {@link Configuration} to examine to determine the archive directory
    * @param family the family hosting the store files
    * @param compactedFiles files to be disposed of. No further reading of these files should be
    *          attempted; otherwise likely to cause an {@link IOException}
    * @throws IOException if the files could not be correctly disposed.
    */
-  public static void archiveStoreFiles(FileSystem fs, HRegion parent,
-      Configuration conf, byte[] family, Collection<StoreFile> compactedFiles) throws IOException {
+  public static void archiveStoreFiles(Configuration conf, FileSystem fs, HRegion parent,
+      byte[] family, Collection<StoreFile> compactedFiles) throws IOException {
 
     // sometimes in testing, we don't have rss, so we need to check for that
     if (fs == null) {

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java?rev=1445841&r1=1445840&r2=1445841&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java Wed Feb 13 18:49:44 2013
@@ -245,7 +245,7 @@ class CatalogJanitor extends Chore {
       }
       FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
       LOG.debug("Archiving parent region:" + parent);
-      HFileArchiver.archiveRegion(fs, parent);
+      HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, parent);
       MetaEditor.deleteRegion(this.server.getCatalogTracker(), parent);
       result = true;
     }

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java?rev=1445841&r1=1445840&r2=1445841&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java Wed Feb 13 18:49:44 2013
@@ -251,13 +251,13 @@ public class MasterFileSystem {
       }
     } while (retrySplitting);
   }
-  
+
   public void splitLog(final ServerName serverName) throws IOException {
     List<ServerName> serverNames = new ArrayList<ServerName>();
     serverNames.add(serverName);
     splitLog(serverNames);
   }
-  
+
   public void splitLog(final List<ServerName> serverNames) throws IOException {
     long splitTime = 0, splitLogSize = 0;
     List<Path> logDirs = new ArrayList<Path>();
@@ -293,7 +293,7 @@ public class MasterFileSystem {
         // splitLogLock ensures that dead region servers' logs are processed
         // one at a time
         this.splitLogLock.lock();
-        try {              
+        try {
           HLogSplitter splitter = HLogSplitter.createLogSplitter(
             conf, rootdir, logDir, oldLogDir, this.fs);
           try {
@@ -443,7 +443,7 @@ public class MasterFileSystem {
 
 
   public void deleteRegion(HRegionInfo region) throws IOException {
-    HFileArchiver.archiveRegion(fs, region);
+    HFileArchiver.archiveRegion(conf, fs, region);
   }
 
   public void deleteTable(byte[] tableName) throws IOException {
@@ -481,7 +481,7 @@ public class MasterFileSystem {
 
   /**
    * Create new HTableDescriptor in HDFS.
-   * 
+   *
    * @param htableDescriptor
    */
   public void createTableDescriptor(HTableDescriptor htableDescriptor)

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1445841&r1=1445840&r2=1445841&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Wed Feb 13 18:49:44 2013
@@ -4310,11 +4310,11 @@ public class HRegion implements HeapSize
     }
 
     // delete out the 'A' region
-    HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(a.getBaseConf()), a.getTableDir(),
-      a.getRegionDir());
+    HFileArchiver.archiveRegion(a.getBaseConf(), fs,
+      FSUtils.getRootDir(a.getBaseConf()), a.getTableDir(), a.getRegionDir());
     // delete out the 'B' region
-    HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(b.getBaseConf()), b.getTableDir(),
-      b.getRegionDir());
+    HFileArchiver.archiveRegion(a.getBaseConf(), fs,
+      FSUtils.getRootDir(b.getBaseConf()), b.getTableDir(), b.getRegionDir());
 
     LOG.info("merge completed. New region is " + dstRegion);
 

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java?rev=1445841&r1=1445840&r2=1445841&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java Wed Feb 13 18:49:44 2013
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
-import java.lang.reflect.InvocationTargetException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.NavigableSet;
-import java.util.Random;
 import java.util.SortedSet;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
@@ -64,7 +62,10 @@ import org.apache.hadoop.hbase.io.hfile.
 import org.apache.hadoop.hbase.io.hfile.InvalidHFileException;
 import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.regionserver.compactions.*;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactSelection;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -1314,7 +1315,7 @@ public class HStore implements Store, St
 
       // let the archive util decide if we should archive or delete the files
       LOG.debug("Removing store files after compaction...");
-      HFileArchiver.archiveStoreFiles(this.fs, this.region, this.conf,
+      HFileArchiver.archiveStoreFiles(this.conf, this.fs, this.region,
         this.family.getName(), compactedFiles);
 
     } catch (IOException e) {

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java?rev=1445841&r1=1445840&r2=1445841&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java Wed Feb 13 18:49:44 2013
@@ -132,7 +132,7 @@ public class TestHFileArchiving {
     // now attempt to depose the region
     Path regionDir = HRegion.getRegionDir(region.getTableDir().getParent(), region.getRegionInfo());
 
-    HFileArchiver.archiveRegion(fs, region.getRegionInfo());
+    HFileArchiver.archiveRegion(UTIL.getConfiguration(), fs, region.getRegionInfo());
 
     // check for the existence of the archive directory and some files in it
     Path archiveDir = HFileArchiveTestingUtil.getRegionArchiveDir(UTIL.getConfiguration(), region);
@@ -192,7 +192,7 @@ public class TestHFileArchiving {
     }
 
     // then archive the region
-    HFileArchiver.archiveRegion(fs, region.getRegionInfo());
+    HFileArchiver.archiveRegion(UTIL.getConfiguration(), fs, region.getRegionInfo());
 
     // and check to make sure the region directoy got deleted
     assertFalse("Region directory (" + regionDir + "), still exists.", fs.exists(regionDir));

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1445841&r1=1445840&r2=1445841&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Wed Feb 13 18:49:44 2013
@@ -30,7 +30,6 @@ import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-import com.google.protobuf.Service;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -73,6 +72,7 @@ import org.junit.experimental.categories
 import org.mockito.Mockito;
 
 import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
 
 @Category(SmallTests.class)
@@ -225,7 +225,7 @@ public class TestCatalogJanitor {
 
     @Override
     public Configuration getConfiguration() {
-      return null;
+      return mfs.conf;
     }
 
     @Override