You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ji...@apache.org on 2008/06/04 06:54:48 UTC

svn commit: r662975 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/client/ src/test/org/apache/hadoop/hbase/client/

Author: jimk
Date: Tue Jun  3 21:54:47 2008
New Revision: 662975

URL: http://svn.apache.org/viewvc?rev=662975&view=rev
Log:
HBASE-639   Add HBaseAdmin.getTableDescriptor function
HBASE-632   HTable.getMetadata is very inefficient
HBASE-654   API HTable.getMetadata().addFamily shouldn't be exposed to user

Added:
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConstants.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnection.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/MetaScanner.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=662975&r1=662974&r2=662975&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Tue Jun  3 21:54:47 2008
@@ -38,6 +38,7 @@
    HBASE-663   Incorrect sequence number for cache flush
    HBASE-655   Need programmatic way to add column family: need programmatic way
                to enable/disable table
+   HBASE-654   API HTable.getMetadata().addFamily shouldn't be exposed to user
    
   IMPROVEMENTS
    HBASE-559   MR example job to count table rows
@@ -71,7 +72,11 @@
                (Clint Morgan via Stack)
    HBASE-579   Add hadoop 0.17.x
    HBASE-660   [Migration] addColumn/deleteColumn functionality in MetaUtils
+   HBASE-632   HTable.getMetadata is very inefficient
 
+  NEW FEATURES
+   HBASE-639   Add HBaseAdmin.getTableDescriptor function
+   
 Release 0.1.2 - 05/13/2008
    
   BUG FIXES

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConstants.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConstants.java?rev=662975&r1=662974&r2=662975&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConstants.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConstants.java Tue Jun  3 21:54:47 2008
@@ -31,6 +31,7 @@
   static final Long ZERO_L = Long.valueOf(0L);
   
   static final String NINES = "99999999999999";
+  static final String ZEROES = "00000000000000";
   
   // For migration
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java?rev=662975&r1=662974&r2=662975&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java Tue Jun  3 21:54:47 2008
@@ -182,6 +182,7 @@
    * descriptors.
    * @see #getNameAsString()
    */
+  @Override
   public String toString() {
     return "name: " + Bytes.toString(this.name) + ", families: " +
       this.families.values();

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnection.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnection.java?rev=662975&r1=662974&r2=662975&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnection.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnection.java Tue Jun  3 21:54:47 2008
@@ -61,6 +61,14 @@
   public HTableDescriptor[] listTables() throws IOException;
   
   /**
+   * @param tableName
+   * @return table metadata 
+   * @throws IOException
+   */
+  public HTableDescriptor getHTableDescriptor(byte[] tableName)
+  throws IOException;
+  
+  /**
    * Find the location of the region of <i>tableName</i> that <i>row</i>
    * lives in.
    * @param tableName name of the table <i>row</i> is in

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=662975&r1=662974&r2=662975&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java Tue Jun  3 21:54:47 2008
@@ -202,9 +202,8 @@
       if (this.master == null) {
         if (masterLocation == null) {
           throw new MasterNotRunningException();
-        } else {
-          throw new MasterNotRunningException(masterLocation.toString());
         }
+        throw new MasterNotRunningException(masterLocation.toString());
       }
       return this.master;
     }
@@ -267,8 +266,11 @@
 
       MetaScannerVisitor visitor = new MetaScannerVisitor() {
 
-        public boolean processRow(RowResult rowResult,
-            HRegionLocation metaLocation, HRegionInfo info) throws IOException {
+        /** {@inheritDoc} */
+        public boolean processRow(
+            @SuppressWarnings("unused") RowResult rowResult,
+            @SuppressWarnings("unused") HRegionLocation metaLocation,
+            HRegionInfo info) {
 
           // Only examine the rows where the startKey is zero length
           if (info.getStartKey().length == 0) {
@@ -283,6 +285,40 @@
       return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
     }
 
+    /**
+     * @param tableName
+     * @return table metadata 
+     * @throws IOException
+     */
+    public HTableDescriptor getHTableDescriptor(byte[] tableName)
+    throws IOException {
+      if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
+        return new UnmodifyableHTableDescriptor(HTableDescriptor.ROOT_TABLEDESC);
+      }
+      HTable meta = new HTable(conf,
+          Bytes.equals(tableName, HConstants.META_TABLE_NAME) ?
+              HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME);
+      Scanner s = meta.getScanner(HConstants.COL_REGIONINFO_ARRAY,
+          HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES));
+      try {
+        RowResult r = null;
+        while ((r = s.next()) != null) {
+          Cell c = r.get(HConstants.COL_REGIONINFO);
+          if (c != null) {
+            HRegionInfo info = Writables.getHRegionInfoOrNull(c.getValue());
+            if (info != null) {
+              if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
+                return new UnmodifyableHTableDescriptor(info.getTableDesc());
+              }
+            }
+          }
+        }
+        return null;
+      } finally {
+        s.close();
+      }
+    }
+
     /** {@inheritDoc} */
     public HRegionLocation locateRegion(final byte [] tableName,
         final byte [] row)

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java?rev=662975&r1=662974&r2=662975&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java Tue Jun  3 21:54:47 2008
@@ -270,7 +270,9 @@
       // The root region is always online
       return false;
     }
-    HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
+    HTable meta = new HTable(conf,
+        Bytes.equals(tableName, HConstants.META_TABLE_NAME) ?
+            HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME);
     Scanner s = meta.getScanner(HConstants.COL_REGIONINFO_ARRAY,
         HRegionInfo.createRegionName(tableName, null, HConstants.NINES));
     try {
@@ -336,20 +338,12 @@
   }
 
   /**
-   * TODO: Make the return read-only.
    * @return table metadata 
    * @throws IOException
    */
+  @Deprecated
   public HTableDescriptor getMetadata() throws IOException {
-    HTableDescriptor [] metas = this.connection.listTables();
-    HTableDescriptor result = null;
-    for (int i = 0; i < metas.length; i++) {
-      if (Bytes.equals(metas[i].getName(), this.tableName)) {
-        result = metas[i];
-        break;
-      }
-    }
-    return result;
+    return this.connection.getHTableDescriptor(this.tableName);
   }
 
   /**
@@ -388,16 +382,15 @@
    * @return A map of HRegionInfo with it's server address
    * @throws IOException
    */
-  @SuppressWarnings("null")
   public Map<HRegionInfo, HServerAddress> getRegionsInfo() throws IOException {
     final HashMap<HRegionInfo, HServerAddress> regionMap =
       new HashMap<HRegionInfo, HServerAddress>();
 
     MetaScannerVisitor visitor = new MetaScannerVisitor() {
-      @SuppressWarnings("unused")
       public boolean processRow(@SuppressWarnings("unused") RowResult rowResult,
-          HRegionLocation metaLocation, HRegionInfo info)
-      throws IOException {
+          HRegionLocation metaLocation, HRegionInfo hri) {
+        
+        HRegionInfo info = new UnmodifyableHRegionInfo(hri); 
         if (!(Bytes.equals(info.getTableDesc().getName(), getTableName()))) {
           return false;
         }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/MetaScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/MetaScanner.java?rev=662975&r1=662974&r2=662975&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/MetaScanner.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/MetaScanner.java Tue Jun  3 21:54:47 2008
@@ -89,6 +89,7 @@
      * 
      * @param rowResult
      * @param metaLocation
+     * @param info
      * @return A boolean to know if it should continue to loop in the region
      * @throws IOException
      */

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java?rev=662975&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java Tue Jun  3 21:54:47 2008
@@ -0,0 +1,87 @@
+/**
+ * Copyright 2008 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+
+class UnmodifyableHRegionInfo extends HRegionInfo {
+  /* Default constructor - creates empty object */
+  UnmodifyableHRegionInfo() {
+    super(new UnmodifyableHTableDescriptor(), null, null);
+  }
+  
+  /*
+   * Construct HRegionInfo with explicit parameters
+   * 
+   * @param tableDesc the table descriptor
+   * @param startKey first key in region
+   * @param endKey end of key range
+   * @throws IllegalArgumentException
+   */
+  UnmodifyableHRegionInfo(final HTableDescriptor tableDesc,
+      final byte [] startKey, final byte [] endKey)
+  throws IllegalArgumentException {
+    super(new UnmodifyableHTableDescriptor(tableDesc), startKey, endKey, false);
+  }
+
+  /*
+   * Construct HRegionInfo with explicit parameters
+   * 
+   * @param tableDesc the table descriptor
+   * @param startKey first key in region
+   * @param endKey end of key range
+   * @param split true if this region has split and we have daughter regions
+   * regions that may or may not hold references to this region.
+   * @throws IllegalArgumentException
+   */
+  UnmodifyableHRegionInfo(HTableDescriptor tableDesc,
+      final byte [] startKey, final byte [] endKey, final boolean split)
+  throws IllegalArgumentException {
+    super(new UnmodifyableHTableDescriptor(tableDesc), startKey, endKey, split);
+  }
+  
+  /*
+   * Creates an unmodifyable copy of an HRegionInfo
+   * 
+   * @param info
+   */
+  UnmodifyableHRegionInfo(HRegionInfo info) {
+    super(new UnmodifyableHTableDescriptor(info.getTableDesc()),
+        info.getStartKey(), info.getEndKey(), info.isSplit());
+  }
+  
+  /**
+   * @param split set split status
+   */
+  @Override
+  public void setSplit(boolean split) {
+    throw new UnsupportedOperationException("HRegionInfo is read-only");
+  }
+
+  /**
+   * @param offLine set online - offline status
+   */
+  @Override
+  public void setOffline(boolean offLine) {
+    throw new UnsupportedOperationException("HRegionInfo is read-only");
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java?rev=662975&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java Tue Jun  3 21:54:47 2008
@@ -0,0 +1,89 @@
+/**
+ * Copyright 2008 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.util.Bytes;
+
+class UnmodifyableHTableDescriptor extends HTableDescriptor {
+  /*
+   * Constructs an empty object.
+   * For deserializing an HTableDescriptor instance only.
+   */
+  UnmodifyableHTableDescriptor() {
+    super();
+  }
+
+  /*
+   * Constructor.
+   * @param name Table name.
+   * @throws IllegalArgumentException if passed a table name
+   * that is made of other than 'word' characters, underscore or period: i.e.
+   * <code>[a-zA-Z_0-9.].
+   * @see <a href="HADOOP-1581">HADOOP-1581 HBASE: Un-openable tablename bug</a>
+   */
+  UnmodifyableHTableDescriptor(final String name) {
+    this(Bytes.toBytes(name));
+  }
+
+  /*
+   * Constructor.
+   * @param name Table name.
+   * @throws IllegalArgumentException if passed a table name
+   * that is made of other than 'word' characters, underscore or period: i.e.
+   * <code>[a-zA-Z_0-9.].
+   * @see <a href="HADOOP-1581">HADOOP-1581 HBASE: Un-openable tablename bug</a>
+   */
+  UnmodifyableHTableDescriptor(final byte [] name) {
+    super(name);
+  }
+  
+  /*
+   * Create an unmodifyable copy of an HTableDescriptor
+   * @param desc
+   */
+  UnmodifyableHTableDescriptor(final HTableDescriptor desc) {
+    super(desc.getName());
+    for (HColumnDescriptor c: desc.getFamilies()) {
+      super.addFamily(c);
+    }
+  }
+
+  /**
+   * Does NOT add a column family. This object is immutable
+   * @param family HColumnDescriptor of familyto add.
+   */
+  @Override
+  public void addFamily(final HColumnDescriptor family) {
+    throw new UnsupportedOperationException("HTableDescriptor is read-only");
+  }
+
+  /**
+   * @param column
+   * @return Column descriptor for the passed family name or the family on
+   * passed in column.
+   */
+  @Override
+  public HColumnDescriptor removeFamily(final byte [] column) {
+    throw new UnsupportedOperationException("HTableDescriptor is read-only");
+  }
+}

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java?rev=662975&r1=662974&r2=662975&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java Tue Jun  3 21:54:47 2008
@@ -22,8 +22,6 @@
 import java.io.IOException;
 import java.util.Map;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseClusterTestCase;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -39,7 +37,6 @@
  * Tests HTable
  */
 public class TestHTable extends HBaseClusterTestCase implements HConstants {
-  private static final Log LOG = LogFactory.getLog(TestHTable.class);
   private static final HColumnDescriptor column =
     new HColumnDescriptor(COLUMN_FAMILY);
 
@@ -84,7 +81,8 @@
     HTable a = new HTable(conf, tableAname);
     
     // Assert the metadata is good.
-    HTableDescriptor meta = a.getMetadata();
+    HTableDescriptor meta =
+      a.getConnection().getHTableDescriptor(tableAdesc.getName());
     assertTrue(meta.equals(tableAdesc));
     
     BatchUpdate batchUpdate = new BatchUpdate(row);