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 2009/04/30 21:00:53 UTC

svn commit: r770397 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/regionserver/ src/test/org/apache/hadoop/hbase/regionserver/

Author: stack
Date: Thu Apr 30 19:00:52 2009
New Revision: 770397

URL: http://svn.apache.org/viewvc?rev=770397&view=rev
Log:
HBASE-1322 hbase-1234 broke TestAtomicIncrement; fix and reenable

Added:
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestAtomicIncrement.java
Removed:
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/DisabledTestAtomicIncrement.java
Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=770397&r1=770396&r2=770397&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Thu Apr 30 19:00:52 2009
@@ -96,6 +96,8 @@
                each access
    HBASE-1358  Bug in reading from Memcache method (read only from snapshot)
                (Evgeny Ryabitskiy via Stack)
+   HBASE-1322  hbase-1234 broke TestAtomicIncrement; fix and reenable
+               (Evgeny Ryabitskiy and Ryan Rawson via Stack)
 
   IMPROVEMENTS
    HBASE-1089  Add count of regions on filesystem to master UI; add percentage

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=770397&r1=770396&r2=770397&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 Thu Apr 30 19:00:52 2009
@@ -465,8 +465,6 @@
    * @return true if the table contains the specified family name
    */
   public boolean hasFamily(final byte [] c) {
-    // If index is -1, then presume we were passed a column family name minus
-    // the colon delimiter.
     return families.containsKey(c);
   }
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java?rev=770397&r1=770396&r2=770397&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java Thu Apr 30 19:00:52 2009
@@ -191,6 +191,11 @@
     this.lock.readLock().lock();
     try {
       boolean notpresent = this.memcache.add(kv);
+      // if false then memcache is not changed (look memcache.add(kv) docs)
+      // need to remove kv and add again to replace it
+      if (!notpresent && this.memcache.remove(kv)) {
+        this.memcache.add(kv);
+      }
       size = heapSize(kv, notpresent);
     } finally {
       this.lock.readLock().unlock();

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestAtomicIncrement.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestAtomicIncrement.java?rev=770397&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestAtomicIncrement.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestAtomicIncrement.java Thu Apr 30 19:00:52 2009
@@ -0,0 +1,118 @@
+/**
+ * Copyright 2007 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.regionserver;
+
+import java.io.IOException;
+
+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;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public class TestAtomicIncrement extends HBaseClusterTestCase {
+  static final Log LOG = LogFactory.getLog(TestAtomicIncrement.class);
+
+  private static final byte [] CONTENTS = Bytes.toBytes("contents:");
+
+  public void testIncrement() throws IOException {
+    try {
+      HTable table = null;
+
+      // Setup
+
+      HTableDescriptor desc = new HTableDescriptor(getName());
+      desc.addFamily(
+          new HColumnDescriptor(CONTENTS,               // Column name
+              1,                                        // Max versions
+              HColumnDescriptor.DEFAULT_COMPRESSION,   // no compression
+              HColumnDescriptor.DEFAULT_IN_MEMORY,      // not in memory
+              HColumnDescriptor.DEFAULT_BLOCKCACHE,
+              HColumnDescriptor.DEFAULT_LENGTH,
+              HColumnDescriptor.DEFAULT_TTL,
+              false
+          )
+      );
+
+      // Create the table
+
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      admin.createTable(desc);
+
+      try {
+        // Give cache flusher and log roller a chance to run
+        // Otherwise we'll never hit the bloom filter, just the memcache
+        Thread.sleep(conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000) * 10);
+        
+      } catch (InterruptedException e) {
+        // ignore
+      }
+      // Open table
+
+      table = new HTable(conf, desc.getName());
+      
+      byte [] row = Bytes.toBytes("foo");
+      byte [] column = "contents:1".getBytes(HConstants.UTF8_ENCODING);
+      // increment by 1:
+      assertEquals(1L, table.incrementColumnValue(row, column, 1));
+      
+      // set a weird value, then increment:
+      row = Bytes.toBytes("foo2");
+      byte [] value = {0,0,2};
+      BatchUpdate bu = new BatchUpdate(row);
+      bu.put(column, value);
+      table.commit(bu);
+      
+      assertEquals(3L, table.incrementColumnValue(row, column, 1));
+
+      assertEquals(-2L, table.incrementColumnValue(row, column, -5));
+
+      row = Bytes.toBytes("foo3");
+      byte[] value2 = {1,2,3,4,5,6,7,8,9};
+      bu = new BatchUpdate(row);
+      bu.put(column, value2);
+      table.commit(bu);
+      
+      try {
+        table.incrementColumnValue(row, column, 1);
+        fail();
+      } catch (IOException e) {
+        System.out.println("Expected exception: " + e);
+        // expected exception.
+      }
+      
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      if (e instanceof IOException) {
+        IOException i = (IOException) e;
+        throw i;
+      }
+      fail();
+    }
+
+  }
+
+}