You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2013/01/04 01:37:16 UTC

svn commit: r1428684 - in /hbase/branches/0.94/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/java/org/apache/hadoo...

Author: enis
Date: Fri Jan  4 00:37:16 2013
New Revision: 1428684

URL: http://svn.apache.org/viewvc?rev=1428684&view=rev
Log:
HBASE-7423. HFileArchiver should not use the configuration from the Filesystem

Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java?rev=1428684&r1=1428683&r2=1428684&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java Fri Jan  4 00:37:16 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/0.94/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java?rev=1428684&r1=1428683&r2=1428684&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java Fri Jan  4 00:37:16 2013
@@ -214,7 +214,7 @@ class CatalogJanitor extends Chore {
 
   /**
    * If daughters no longer hold reference to the parents, delete the parent.
-   * @param server HRegionInterface of meta server to talk to 
+   * @param server HRegionInterface of meta server to talk to
    * @param parent HRegionInfo of split offlined parent
    * @param rowContent Content of <code>parent</code> row in
    * <code>metaRegionName</code>
@@ -246,7 +246,7 @@ class CatalogJanitor extends Chore {
         this.services.getAssignmentManager().regionOffline(parent);
       }
       FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
-      HFileArchiver.archiveRegion(fs, parent);
+      HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, parent);
       MetaEditor.deleteRegion(this.server.getCatalogTracker(), parent);
       result = true;
     }
@@ -291,7 +291,7 @@ class CatalogJanitor extends Chore {
   /**
    * Checks if a daughter region -- either splitA or splitB -- still holds
    * references to parent.
-   * @param parent Parent region name. 
+   * @param parent Parent region name.
    * @param split Which column family.
    * @param qualifier Which of the daughters to look at, splitA or splitB.
    * @return A pair where the first boolean says whether or not the daughter

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java?rev=1428684&r1=1428683&r2=1428684&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java Fri Jan  4 00:37:16 2013
@@ -247,13 +247,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>();
@@ -279,7 +279,7 @@ public class MasterFileSystem {
       LOG.info("No logs to split");
       return;
     }
-      
+
     if (distributedLogSplitting) {
       splitLogManager.handleDeadWorkers(serverNames);
       splitTime = EnvironmentEdgeManager.currentTimeMillis();
@@ -290,7 +290,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 {
@@ -344,7 +344,7 @@ public class MasterFileSystem {
         // there is one datanode it will succeed. Permission problems should have
         // already been caught by mkdirs above.
         FSUtils.setVersion(fs, rd, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
-          10 * 1000), c.getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS, 
+          10 * 1000), c.getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS,
         		  HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
       } else {
         if (!fs.isDirectory(rd)) {
@@ -352,7 +352,7 @@ public class MasterFileSystem {
         }
         // as above
         FSUtils.checkVersion(fs, rd, true, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
-          10 * 1000), c.getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS, 
+          10 * 1000), c.getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS,
         		  HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
       }
     } catch (IllegalArgumentException iae) {
@@ -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/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1428684&r1=1428683&r2=1428684&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Fri Jan  4 00:37:16 2013
@@ -4275,10 +4275,10 @@ public class HRegion implements HeapSize
     }
 
     // delete out the 'A' region
-    HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(a.getConf()), a.getTableDir(),
+    HFileArchiver.archiveRegion(a.getConf(), fs, FSUtils.getRootDir(a.getConf()), a.getTableDir(),
       a.getRegionDir());
     // delete out the 'B' region
-    HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(b.getConf()), b.getTableDir(),
+    HFileArchiver.archiveRegion(b.getConf(), fs, FSUtils.getRootDir(b.getConf()), b.getTableDir(),
       b.getRegionDir());
 
     LOG.info("merge completed. New region is " + dstRegion);

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1428684&r1=1428683&r2=1428684&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Fri Jan  4 00:37:16 2013
@@ -584,7 +584,7 @@ public class Store extends SchemaConfigu
     StoreFile.Reader r = sf.createReader();
     this.storeSize += r.length();
     this.totalUncompressedBytes += r.getTotalUncompressedBytes();
-    
+
     LOG.info("Moved hfile " + srcPath + " into store directory " +
         homedir + " - updating store file list.");
 
@@ -1626,7 +1626,7 @@ public class Store extends SchemaConfigu
 
       // 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, this.family.getName(),
+      HFileArchiver.archiveStoreFiles(this.conf, this.fs, this.region, this.family.getName(),
         compactedFiles);
 
     } catch (IOException e) {
@@ -2076,10 +2076,10 @@ public class Store extends SchemaConfigu
 
   boolean throttleCompaction(long compactionSize) {
     long throttlePoint = conf.getLong(
-        "hbase.regionserver.thread.compaction.throttle",  
-        2 * this.minFilesToCompact * this.region.memstoreFlushSize);  
-    return compactionSize > throttlePoint;  
-  } 
+        "hbase.regionserver.thread.compaction.throttle",
+        2 * this.minFilesToCompact * this.region.memstoreFlushSize);
+    return compactionSize > throttlePoint;
+  }
 
   public HRegion getHRegion() {
     return this.region;

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java?rev=1428684&r1=1428683&r2=1428684&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java Fri Jan  4 00:37:16 2013
@@ -137,7 +137,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);
@@ -197,7 +197,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/0.94/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1428684&r1=1428683&r2=1428684&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Fri Jan  4 00:37:16 2013
@@ -68,7 +68,6 @@ import org.apache.hadoop.hbase.util.HFil
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
@@ -129,7 +128,7 @@ public class TestCatalogJanitor {
     public void abort(String why, Throwable e) {
       //no-op
     }
-    
+
     @Override
     public boolean isAborted() {
       return false;
@@ -206,7 +205,7 @@ public class TestCatalogJanitor {
 
     @Override
     public Configuration getConfiguration() {
-      return null;
+      return mfs.conf;
     }
 
     @Override
@@ -218,7 +217,7 @@ public class TestCatalogJanitor {
     public void abort(String why, Throwable e) {
       //no-op
     }
-    
+
     @Override
     public boolean isAborted() {
       return false;