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 2009/04/10 21:36:08 UTC

svn commit: r764008 - in /hadoop/hbase/branches/0.19: CHANGES.txt src/java/org/apache/hadoop/hbase/io/Cell.java src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java

Author: jimk
Date: Fri Apr 10 19:36:07 2009
New Revision: 764008

URL: http://svn.apache.org/viewvc?rev=764008&view=rev
Log:
HBASE-1202  getRow does not always work when specifying number of versions

Added:
    hadoop/hbase/branches/0.19/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java
Modified:
    hadoop/hbase/branches/0.19/CHANGES.txt
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/io/Cell.java

Modified: hadoop/hbase/branches/0.19/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/CHANGES.txt?rev=764008&r1=764007&r2=764008&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.19/CHANGES.txt Fri Apr 10 19:36:07 2009
@@ -1,4 +1,11 @@
 HBase Change Log
+
+Unreleased changes
+  BUG FIXES
+   HBASE-1303  Secondary index configuration prevents HBase from starting
+               (Ken Weiner via Stack)
+   HBASE-1202  getRow does not always work when specifying number of versions
+
 Release 0.19.1 - 03/19/2009
   BUG FIXES
    HBASE-1121  Cluster confused about where -ROOT- is
@@ -28,8 +35,6 @@
    HBASE-1220  Don't reopen file if already open when updating readers
                underneath scanners
    HBASE-1256  NPE in StoreFileScanner durring cache flush
-   HBASE-1303  Secondary index configuration prevents HBase from starting
-               (Ken Weiner via Stack)
 
 
   IMPROVEMENTS

Modified: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/io/Cell.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/io/Cell.java?rev=764008&r1=764007&r2=764008&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/io/Cell.java (original)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/io/Cell.java Fri Apr 10 19:36:07 2009
@@ -104,19 +104,16 @@
     return valueMap.size();
   }
   
-  /** Add values and timestamps of another cell into this cell 
-   * @param c Cell
-   */
-  public void mergeCell(Cell c) {
-    valueMap.putAll(c.valueMap);
-  }
-  
-  /** Add a new timestamp and value to this cell
+  /** 
+   * Add a new timestamp and value to this cell, provided timestamp does not
+   * already exist.
    * @param val value
    * @param ts timestamp
    */
   public void add(byte[] val, long ts) {
-    valueMap.put(ts, val);
+    if (!valueMap.containsKey(ts)) {
+      valueMap.put(ts, val);
+    }
   }
   
   @Override

Added: hadoop/hbase/branches/0.19/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java?rev=764008&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java (added)
+++ hadoop/hbase/branches/0.19/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java Fri Apr 10 19:36:07 2009
@@ -0,0 +1,103 @@
+/**
+ * Copyright 2009 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 java.util.Iterator;
+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;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * 
+ */
+public class TestGetRowVersions extends HBaseClusterTestCase {
+  private static final Log LOG = LogFactory.getLog(TestGetRowVersions.class);
+  private static final String TABLE_NAME = "test";
+  private static final String CONTENTS_STR = "contents:";
+  private static final String ROW = "row";
+  private static final String COLUMN = "contents:contents";
+  private static final long TIMESTAMP = System.currentTimeMillis();
+  private static final String VALUE1 = "value1";
+  private static final String VALUE2 = "value2";
+  private HBaseAdmin admin = null;
+  private HTable table = null;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
+    desc.addFamily(new HColumnDescriptor(CONTENTS_STR));
+    this.admin = new HBaseAdmin(conf);
+    this.admin.createTable(desc);
+    this.table = new HTable(conf, TABLE_NAME);
+  }
+
+  /** @throws Exception */
+  public void testGetRowMultipleVersions() throws Exception {
+    BatchUpdate b = new BatchUpdate(ROW, TIMESTAMP);
+    b.put(COLUMN, Bytes.toBytes(VALUE1));
+    this.table.commit(b);
+    // Shut down and restart the HBase cluster
+    this.cluster.shutdown();
+    LOG.debug("HBase cluster shut down -- restarting");
+    this.hBaseClusterSetup();
+    // Make a new connection
+    this.table = new HTable(conf, TABLE_NAME);
+    // Overwrite previous value
+    b = new BatchUpdate(ROW, TIMESTAMP);
+    b.put(COLUMN, Bytes.toBytes(VALUE2));
+    this.table.commit(b);
+    // Now verify that getRow(row, column, latest) works
+    RowResult r = table.getRow(ROW);
+    assertNotNull(r);
+    assertTrue(r.size() != 0);
+    Cell c = r.get(COLUMN);
+    assertNotNull(c);
+    assertTrue(c.getValue().length != 0);
+    String value = Bytes.toString(c.getValue());
+    assertTrue(value.compareTo(VALUE2) == 0);
+    // Now check getRow with multiple versions
+    r = table.getRow(ROW, HConstants.ALL_VERSIONS);
+    for (Map.Entry<byte[], Cell> e: r.entrySet()) {
+      // Column name
+//      System.err.print("  " + Bytes.toString(e.getKey()));
+      c = e.getValue();
+      
+      // Need to iterate since there may be multiple versions
+      for (Iterator<Map.Entry<Long, byte[]>> it = c.iterator();
+            it.hasNext(); ) {
+        Map.Entry<Long, byte[]> v = it.next();
+        value = Bytes.toString(v.getValue());
+//        System.err.println(" = " + value);
+        assertTrue(VALUE2.compareTo(Bytes.toString(v.getValue())) == 0);
+      }
+    }
+  }
+}