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

svn commit: r934641 - in /hadoop/hbase/branches/0.20_pre_durability: bin/add_table.rb bin/set_meta_block_caching.rb src/java/org/apache/hadoop/hbase/HTableDescriptor.java src/java/org/apache/hadoop/hbase/master/HMaster.java

Author: stack
Date: Thu Apr 15 23:33:51 2010
New Revision: 934641

URL: http://svn.apache.org/viewvc?rev=934641&view=rev
Log:
HBASE-2451 .META. by-passes cache; BLOCKCACHE=>'false'

Added:
    hadoop/hbase/branches/0.20_pre_durability/bin/set_meta_block_caching.rb
Modified:
    hadoop/hbase/branches/0.20_pre_durability/bin/add_table.rb
    hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
    hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/master/HMaster.java

Modified: hadoop/hbase/branches/0.20_pre_durability/bin/add_table.rb
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/bin/add_table.rb?rev=934641&r1=934640&r2=934641&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/bin/add_table.rb (original)
+++ hadoop/hbase/branches/0.20_pre_durability/bin/add_table.rb Thu Apr 15 23:33:51 2010
@@ -1,6 +1,5 @@
 # Script adds a table back to a running hbase.
-# Currently only works on a copied aside table.
-# You cannot parse arbitrary table name.
+# Currently only works on if table data is in place.
 # 
 # To see usage for this script, run: 
 #

Added: hadoop/hbase/branches/0.20_pre_durability/bin/set_meta_block_caching.rb
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/bin/set_meta_block_caching.rb?rev=934641&view=auto
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/bin/set_meta_block_caching.rb (added)
+++ hadoop/hbase/branches/0.20_pre_durability/bin/set_meta_block_caching.rb Thu Apr 15 23:33:51 2010
@@ -0,0 +1,82 @@
+# Set in_memory=true and blockcache=true on catalog tables.
+# The .META. and -ROOT- tables can be created with caching and
+# in_memory set to false.  You want them set to true so that
+# these hot tables make it into cache.  To see if the
+# .META. table has BLOCKCACHE set, in the shell do the following:
+#
+#   hbase> scan '-ROOT-'
+#
+# Look for the 'info' column family.  See if BLOCKCACHE => 'true'? 
+# If not, run this script and it will set the value to true.
+# Setting cache to 'true' will only take effect on region restart
+# of if you close the .META. region -- *disruptive* -- and have
+# it deploy elsewhere.  This script runs against an up and running
+# hbase instance.
+# 
+# To see usage for this script, run: 
+#
+#  ${HBASE_HOME}/bin/hbase org.jruby.Main set_meta_block_caching.rb
+#
+include Java
+import org.apache.hadoop.hbase.util.Bytes
+import org.apache.hadoop.hbase.HConstants
+import org.apache.hadoop.hbase.HRegionInfo
+import org.apache.hadoop.hbase.client.HTable
+import org.apache.hadoop.hbase.client.Delete
+import org.apache.hadoop.hbase.client.Put
+import org.apache.hadoop.hbase.client.Scan
+import org.apache.hadoop.hbase.HTableDescriptor
+import org.apache.hadoop.hbase.HBaseConfiguration
+import org.apache.hadoop.hbase.util.FSUtils
+import org.apache.hadoop.hbase.util.Writables
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.FileSystem
+import org.apache.commons.logging.LogFactory
+
+# Name of this script
+NAME = "set_meta_block_caching.rb"
+
+
+# Print usage for this script
+def usage
+  puts 'Usage: %s.rb]' % NAME
+  exit!
+end
+
+# Get configuration to use.
+c = HBaseConfiguration.new()
+
+# Set hadoop filesystem configuration using the hbase.rootdir.
+# Otherwise, we'll always use localhost though the hbase.rootdir
+# might be pointing at hdfs location.
+c.set("fs.default.name", c.get(HConstants::HBASE_DIR))
+fs = FileSystem.get(c)
+
+# Get a logger and a metautils instance.
+LOG = LogFactory.getLog(NAME)
+
+# Check arguments
+if ARGV.size > 0
+  usage
+end
+
+# Clean mentions of table from .META.
+# Scan the .META. and remove all lines that begin with tablename
+metaTable = HTable.new(c, HConstants::ROOT_TABLE_NAME)
+scan = Scan.new()
+scan.addColumn(HConstants::CATALOG_FAMILY, HConstants::REGIONINFO_QUALIFIER);
+scanner = metaTable.getScanner(scan)
+while (result = scanner.next())
+  rowid = Bytes.toString(result.getRow())
+  rowidStr = java.lang.String.new(rowid)
+  LOG.info("Setting BLOCKCACHE and IN_MEMORY on: " + rowidStr);
+  hriValue = result.getValue(HConstants::CATALOG_FAMILY, HConstants::REGIONINFO_QUALIFIER)
+  hri = Writables.getHRegionInfo(hriValue)
+  family = hri.getTableDesc().getFamily(HConstants::CATALOG_FAMILY)
+  family.setBlockCacheEnabled(true)
+  family.setInMemory(true)
+  p = Put.new(result.getRow())
+  p.add(HConstants::CATALOG_FAMILY, HConstants::REGIONINFO_QUALIFIER, Writables.getBytes(hri));
+  metaTable.put(p)
+end
+scanner.close()

Modified: hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/HTableDescriptor.java?rev=934641&r1=934640&r2=934641&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/HTableDescriptor.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/HTableDescriptor.java Thu Apr 15 23:33:51 2010
@@ -645,7 +645,7 @@ ISerializable {
       HConstants.ROOT_TABLE_NAME,
       new HColumnDescriptor[] { new HColumnDescriptor(HConstants.CATALOG_FAMILY,
           10,  // Ten is arbitrary number.  Keep versions to help debuggging.
-          Compression.Algorithm.NONE.getName(), false, true, 8 * 1024,
+          Compression.Algorithm.NONE.getName(), true, true, 8 * 1024,
           HConstants.FOREVER, false) });
   
   /** Table descriptor for <code>.META.</code> catalog table */
@@ -653,7 +653,7 @@ ISerializable {
       HConstants.META_TABLE_NAME, new HColumnDescriptor[] {
           new HColumnDescriptor(HConstants.CATALOG_FAMILY,
             10, // Ten is arbitrary number.  Keep versions to help debuggging.
-            Compression.Algorithm.NONE.getName(), false, true, 8 * 1024,
+            Compression.Algorithm.NONE.getName(), true, true, 8 * 1024,
             HConstants.FOREVER, false),
           new HColumnDescriptor(HConstants.CATALOG_HISTORIAN_FAMILY,
             HConstants.ALL_VERSIONS, Compression.Algorithm.NONE.getName(),

Modified: hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/master/HMaster.java?rev=934641&r1=934640&r2=934641&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/master/HMaster.java Thu Apr 15 23:33:51 2010
@@ -275,20 +275,20 @@ public class HMaster extends Thread impl
       // created here in bootstap and it'll need to be cleaned up.  Better to
       // not make it in first place.  Turn off block caching for bootstrap.
       // Enable after.
-      setBlockCaching(HRegionInfo.ROOT_REGIONINFO, false);
-      setBlockCaching(HRegionInfo.FIRST_META_REGIONINFO, false);
-      HRegion root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO,
-        this.rootdir, this.conf);
-      HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO,
-        this.rootdir, this.conf);
+      HRegionInfo rootHRI = new HRegionInfo(HRegionInfo.ROOT_REGIONINFO);
+      setInfoFamilyCaching(rootHRI, false);
+      HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
+      setInfoFamilyCaching(metaHRI, false);
+      HRegion root = HRegion.createHRegion(rootHRI, this.rootdir, this.conf);
+      HRegion meta = HRegion.createHRegion(metaHRI, this.rootdir, this.conf);
+      setInfoFamilyCaching(rootHRI, true);
+      setInfoFamilyCaching(metaHRI, true);
       // Add first region from the META table to the ROOT region.
       HRegion.addRegionToMETA(root, meta);
       root.close();
       root.getLog().closeAndDelete();
       meta.close();
       meta.getLog().closeAndDelete();
-      setBlockCaching(HRegionInfo.ROOT_REGIONINFO, true);
-      setBlockCaching(HRegionInfo.FIRST_META_REGIONINFO, true);
     } catch (IOException e) {
       e = RemoteExceptionHandler.checkIOException(e);
       LOG.error("bootstrap", e);
@@ -300,9 +300,12 @@ public class HMaster extends Thread impl
    * @param hri Set all family block caching to <code>b</code>
    * @param b
    */
-  private void setBlockCaching(final HRegionInfo hri, final boolean b) {
+  private void setInfoFamilyCaching(final HRegionInfo hri, final boolean b) {
     for (HColumnDescriptor hcd: hri.getTableDesc().families.values()) {
-      hcd.setBlockCacheEnabled(b);
+      if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
+        hcd.setBlockCacheEnabled(b);
+        hcd.setInMemory(b);
+      }
     }
   }