You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nd...@apache.org on 2014/04/23 06:09:15 UTC

svn commit: r1589336 - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-common/src/main/resources/ hbase-server/src/test/java/org/apache/hadoop/hbase/

Author: ndimiduk
Date: Wed Apr 23 04:09:14 2014
New Revision: 1589336

URL: http://svn.apache.org/r1589336
Log:
HBASE-10950 Add a configuration point for MaxVersion of Column Family (Enoch Hsu)

Added:
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
    hbase/trunk/hbase-common/src/main/resources/hbase-default.xml

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java?rev=1589336&r1=1589335&r2=1589336&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java Wed Apr 23 04:09:14 2014
@@ -121,7 +121,8 @@ public class HColumnDescriptor implement
   /**
    * Default number of versions of a record to keep.
    */
-  public static final int DEFAULT_VERSIONS = 1;
+  public static final int DEFAULT_VERSIONS = HBaseConfiguration.create().getInt(
+    "hbase.column.max.version", 1);
 
   /**
    * Default is not to keep a minimum of versions.

Modified: hbase/trunk/hbase-common/src/main/resources/hbase-default.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/resources/hbase-default.xml?rev=1589336&r1=1589335&r2=1589336&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/resources/hbase-default.xml (original)
+++ hbase/trunk/hbase-common/src/main/resources/hbase-default.xml Wed Apr 23 04:09:14 2014
@@ -1001,6 +1001,12 @@ possible configurations would overwhelm 
         dfs.socket.timeout. See the end of HBASE-8389 for more.</description>
   </property>
   <property>
+    <name>hbase.column.max.version</name>
+    <value>1</value>
+    <description>New column family descriptors will use this value as the default number of versions
+      to keep.</description>
+  </property>
+  <property>
     <name>hbase.dfs.client.read.shortcircuit.buffer.size</name>
     <value>131072</value>
     <description>If the DFSClient configuration

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java?rev=1589336&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java Wed Apr 23 04:09:14 2014
@@ -0,0 +1,166 @@
+/**
+ *
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+
+/**
+ * Verify that the HColumnDescriptor version is set correctly by default, hbase-site.xml, and user
+ * input
+ */
+@Category(MediumTests.class)
+public class TestHColumnDescriptorDefaultVersions {
+
+  @Rule
+  public TestName name = new TestName();
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static TableName TABLE_NAME = null;
+  private static final byte[] FAMILY = Bytes.toBytes("cf0");
+
+  /**
+   * Start up a mini cluster and put a small table of empty regions into it.
+   * @throws Exception
+   */
+  @BeforeClass
+  public static void beforeAllTests() throws Exception {
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @Before
+  public void setup() {
+    TABLE_NAME = TableName.valueOf(name.getMethodName());
+
+  }
+
+  @AfterClass
+  public static void afterAllTests() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testCreateTableWithDefault() throws IOException {
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    // Create a table with one family
+    HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME);
+    HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
+    baseHtd.addFamily(hcd);
+    admin.createTable(baseHtd);
+    admin.disableTable(TABLE_NAME);
+    try {
+      // Verify the column descriptor
+      verifyHColumnDescriptor(1, TABLE_NAME, FAMILY);
+    } finally {
+      admin.deleteTable(TABLE_NAME);
+    }
+  }
+
+  @Test
+  public void testCreateTableWithDefaultFromConf() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+    TEST_UTIL.getConfiguration().setInt("hbase.column.max.version", 3);
+    TEST_UTIL.startMiniCluster(1);
+
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    // Create a table with one family
+    HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME);
+    HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
+    hcd.setMaxVersions(TEST_UTIL.getConfiguration().getInt("hbase.column.max.version", 1));
+    baseHtd.addFamily(hcd);
+    admin.createTable(baseHtd);
+    admin.disableTable(TABLE_NAME);
+    try {
+      // Verify the column descriptor
+      verifyHColumnDescriptor(3, TABLE_NAME, FAMILY);
+    } finally {
+      admin.deleteTable(TABLE_NAME);
+    }
+  }
+
+  @Test
+  public void testCreateTableWithSetVersion() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+    TEST_UTIL.getConfiguration().setInt("hbase.column.max.version", 3);
+    TEST_UTIL.startMiniCluster(1);
+
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    // Create a table with one family
+    HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME);
+    HColumnDescriptor hcd =
+        new HColumnDescriptor(FAMILY, 5, HColumnDescriptor.DEFAULT_COMPRESSION,
+            HColumnDescriptor.DEFAULT_IN_MEMORY, HColumnDescriptor.DEFAULT_BLOCKCACHE,
+            HColumnDescriptor.DEFAULT_TTL, HColumnDescriptor.DEFAULT_BLOOMFILTER);
+    baseHtd.addFamily(hcd);
+    admin.createTable(baseHtd);
+    admin.disableTable(TABLE_NAME);
+    try {
+      // Verify the column descriptor
+      verifyHColumnDescriptor(5, TABLE_NAME, FAMILY);
+
+    } finally {
+      admin.deleteTable(TABLE_NAME);
+    }
+  }
+
+  private void verifyHColumnDescriptor(int expected, final TableName tableName,
+      final byte[]... families) throws IOException {
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+
+    // Verify descriptor from master
+    HTableDescriptor htd = admin.getTableDescriptor(tableName);
+    HColumnDescriptor[] hcds = htd.getColumnFamilies();
+    verifyHColumnDescriptor(expected, hcds, tableName, families);
+
+    // Verify descriptor from HDFS
+    MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
+    Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
+    htd = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
+    hcds = htd.getColumnFamilies();
+    verifyHColumnDescriptor(expected, hcds, tableName, families);
+  }
+
+  private void verifyHColumnDescriptor(int expected, final HColumnDescriptor[] hcds,
+      final TableName tableName,
+      final byte[]... families) {
+    for (HColumnDescriptor hcd : hcds) {
+      assertEquals(expected, hcd.getMaxVersions());
+    }
+  }
+
+}
\ No newline at end of file