You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2007/12/09 05:49:58 UTC

svn commit: r602633 - in /lucene/hadoop/trunk/src/contrib/hbase: ./ src/java/org/apache/hadoop/hbase/ src/test/org/apache/hadoop/hbase/

Author: stack
Date: Sat Dec  8 20:49:58 2007
New Revision: 602633

URL: http://svn.apache.org/viewvc?rev=602633&view=rev
Log:
HADOOP-2384 Delete all members of a column family on a specific row

Added:
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDeleteFamily.java
Modified:
    lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java

Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt?rev=602633&r1=602632&r2=602633&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Sat Dec  8 20:49:58 2007
@@ -13,6 +13,8 @@
     HADOOP-2316 Run REST servlet outside of master
                 (Bryan Duxbury & Stack)
     HADOOP-1550 No means of deleting a'row' (Bryan Duxbuery via Stack)
+    HADOOP-2384 Delete all members of a column family on a specific row
+                (Bryan Duxbury via Stack)
 
   OPTIMIZATIONS
 

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java?rev=602633&r1=602632&r2=602633&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java Sat Dec  8 20:49:58 2007
@@ -1275,6 +1275,35 @@
     }
   }
 
+  /**
+   * Delete all cells for a row with matching column family with timestamps
+   * less than or equal to <i>timestamp</i>.
+   *
+   * @param row The row to operate on
+   * @param family The column family to match
+   * @param timestamp Timestamp to match
+   */
+  public void deleteFamily(Text row, Text family, long timestamp)
+  throws IOException{
+    obtainRowLock(row);    
+    
+    try {
+      // find the HStore for the column family
+      LOG.info(family);
+      HStore store = stores.get(HStoreKey.extractFamily(family));
+      // find all the keys that match our criteria
+      List<HStoreKey> keys = store.getKeys(new HStoreKey(row, timestamp), ALL_VERSIONS);
+      
+      // delete all the cells
+      TreeMap<HStoreKey, byte []> edits = new TreeMap<HStoreKey, byte []>();
+      for (HStoreKey key: keys) {
+        edits.put(key, HLogEdit.deleteBytes.get());
+      }
+      update(edits);
+    } finally {
+      releaseRowLock(row);
+    }
+  }
   
   /**
    * Delete one or many cells.

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java?rev=602633&r1=602632&r2=602633&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java Sat Dec  8 20:49:58 2007
@@ -146,6 +146,20 @@
    */
   public void deleteAll(Text regionName, Text row, long timestamp)
   throws IOException;
+
+  /**
+   * Delete all cells for a row with matching column family with timestamps
+   * less than or equal to <i>timestamp</i>.
+   *
+   * @param regionName The name of the region to operate on
+   * @param row The row to operate on
+   * @param family The column family to match
+   * @param timestamp Timestamp to match
+   */
+  public void deleteFamily(Text regionName, Text row, Text family, 
+    long timestamp)
+  throws IOException;
+
   
   //
   // remote scanner interface

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java?rev=602633&r1=602632&r2=602633&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java Sat Dec  8 20:49:58 2007
@@ -1523,6 +1523,13 @@
     region.deleteAll(row, timestamp);
   }
 
+  public void deleteFamily(Text regionName, Text row, Text family, 
+    long timestamp)
+  throws IOException{
+    getRegion(regionName).deleteFamily(row, family, timestamp);
+  }
+
+
   /**
    * @return Info on this server.
    */

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java?rev=602633&r1=602632&r2=602633&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java Sat Dec  8 20:49:58 2007
@@ -789,6 +789,56 @@
     deleteAll(row, HConstants.LATEST_TIMESTAMP);
   }
   
+  /**
+   * Delete all cells for a row with matching column family with timestamps
+   * less than or equal to <i>timestamp</i>.
+   *
+   * @param row The row to operate on
+   * @param family The column family to match
+   * @param timestamp Timestamp to match
+   */
+  public void deleteFamily(final Text row, final Text family, long timestamp)
+  throws IOException {
+    checkClosed();
+    for(int tries = 0; tries < numRetries; tries++) {
+      HRegionLocation r = getRegionLocation(row);
+      HRegionInterface server =
+        connection.getHRegionConnection(r.getServerAddress());
+      try {
+        server.deleteFamily(r.getRegionInfo().getRegionName(), row, family, timestamp);
+        break;
+
+      } catch (IOException e) {
+        if (e instanceof RemoteException) {
+          e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
+        }
+        if (tries == numRetries - 1) {
+          throw e;
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("reloading table servers because: " + e.getMessage());
+        }
+        tableServers = connection.reloadTableServers(tableName);
+      }
+      try {
+        Thread.sleep(this.pause);
+      } catch (InterruptedException x) {
+        // continue
+      }
+    }
+  }
+
+  /**
+   * Delete all cells for a row with matching column family at all timestamps.
+   *
+   * @param row The row to operate on
+   * @param family The column family to match
+   */  
+  public void deleteFamily(final Text row, final Text family)
+  throws IOException{
+    deleteFamily(row, family, HConstants.LATEST_TIMESTAMP);
+  }
+  
   /** 
    * Abort a row mutation.
    * 

Added: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDeleteFamily.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDeleteFamily.java?rev=602633&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDeleteFamily.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDeleteFamily.java Sat Dec  8 20:49:58 2007
@@ -0,0 +1,166 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.io.Text;
+import org.apache.commons.logging.*;
+
+/**
+ * Test the functionality of deleteFamily.
+ */
+public class TestDeleteFamily extends HBaseTestCase {
+  private MiniDFSCluster miniHdfs;
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    this.miniHdfs = new MiniDFSCluster(this.conf, 1, true, null);
+  }
+  
+  /**
+   * Tests for HADOOP-2384.
+   * @throws Exception
+   */
+  public void testDeleteFamily() throws Exception {
+    HRegion region = null;
+    HRegionIncommon region_incommon = null;
+    HLog hlog = new HLog(this.miniHdfs.getFileSystem(), this.testDir,
+      this.conf, null);
+    
+    try{
+      HTableDescriptor htd = createTableDescriptor(getName());
+      HRegionInfo hri = new HRegionInfo(htd, null, null);
+      region = new HRegion(this.testDir, hlog, this.miniHdfs.getFileSystem(),
+        this.conf, hri, null, null);
+      region_incommon = new HRegionIncommon(region);
+      
+      // test memcache
+      makeSureItWorks(region, region_incommon, false);
+      // test hstore
+      makeSureItWorks(region, region_incommon, true);
+      
+    } finally {
+      if (region != null) {
+        try {
+          region.close();
+        } catch (Exception e) {
+          e.printStackTrace();
+        }
+      }
+      hlog.closeAndDelete();
+    }
+  }
+    
+  private void makeSureItWorks(HRegion region, HRegionIncommon region_incommon, 
+    boolean flush)
+  throws Exception{
+    // insert a few versions worth of data for a row
+    Text row = new Text("test_row");
+    long t0 = System.currentTimeMillis();
+    long t1 = t0 - 15000;
+    long t2 = t1 - 15000;
+
+    Text colA = new Text(COLUMNS[0].toString() + "a");
+    Text colB = new Text(COLUMNS[0].toString() + "b");
+    Text colC = new Text(COLUMNS[1].toString() + "c");
+          
+    long lock = region_incommon.startUpdate(row);
+    region_incommon.put(lock, colA, cellData(0, flush).getBytes());
+    region_incommon.put(lock, colB, cellData(0, flush).getBytes());
+    region_incommon.put(lock, colC, cellData(0, flush).getBytes());      
+    region_incommon.commit(lock, t0);
+
+    lock = region_incommon.startUpdate(row);
+    region_incommon.put(lock, colA, cellData(1, flush).getBytes());
+    region_incommon.put(lock, colB, cellData(1, flush).getBytes());
+    region_incommon.put(lock, colC, cellData(1, flush).getBytes());      
+    region_incommon.commit(lock, t1);
+    
+    lock = region_incommon.startUpdate(row);
+    region_incommon.put(lock, colA, cellData(2, flush).getBytes());
+    region_incommon.put(lock, colB, cellData(2, flush).getBytes());
+    region_incommon.put(lock, colC, cellData(2, flush).getBytes());      
+    region_incommon.commit(lock, t2);
+
+    if (flush) {region_incommon.flushcache();}
+
+    // call delete family at a timestamp, make sure only the most recent stuff
+    // for column c is left behind
+    region.deleteFamily(row, COLUMNS[0], t1);
+    if (flush) {region_incommon.flushcache();}
+    // most recent for A,B,C should be fine
+    // A,B at older timestamps should be gone
+    // C should be fine for older timestamps
+    assertCellValueEquals(region, row, colA, t0, cellData(0, flush));
+    assertCellValueEquals(region, row, colA, t1, null);    
+    assertCellValueEquals(region, row, colA, t2, null);
+    assertCellValueEquals(region, row, colB, t0, cellData(0, flush));
+    assertCellValueEquals(region, row, colB, t1, null);
+    assertCellValueEquals(region, row, colB, t2, null);    
+    assertCellValueEquals(region, row, colC, t0, cellData(0, flush));
+    assertCellValueEquals(region, row, colC, t1, cellData(1, flush));
+    assertCellValueEquals(region, row, colC, t2, cellData(2, flush));        
+
+    // call delete family w/o a timestamp, make sure nothing is left except for
+    // column C.
+    region.deleteFamily(row, COLUMNS[0], HConstants.LATEST_TIMESTAMP);
+    if (flush) {region_incommon.flushcache();}
+    // A,B for latest timestamp should be gone
+    // C should still be fine
+    assertCellValueEquals(region, row, colA, t0, null);
+    assertCellValueEquals(region, row, colB, t0, null);
+    assertCellValueEquals(region, row, colC, t0, cellData(0, flush));
+    assertCellValueEquals(region, row, colC, t1, cellData(1, flush));
+    assertCellValueEquals(region, row, colC, t2, cellData(2, flush));        
+    
+  }
+  
+  private void assertCellValueEquals(final HRegion region, final Text row,
+    final Text column, final long timestamp, final String value)
+  throws IOException {
+    Map<Text, byte[]> result = region.getFull(row, timestamp);
+    byte[] cell_value = result.get(column);
+    if(value == null){
+      assertEquals(column.toString() + " at timestamp " + timestamp, null, cell_value);
+    } else {
+      if (cell_value == null) {
+        fail(column.toString() + " at timestamp " + timestamp + 
+          "\" was expected to be \"" + value + " but was null");
+      }
+      assertEquals(column.toString() + " at timestamp " 
+        + timestamp, value, new String(cell_value));
+    }
+  }
+  
+  private String cellData(int tsNum, boolean flush){
+    return "t" + tsNum + " data" + (flush ? " - with flush" : "");
+  }
+  
+  protected void tearDown() throws Exception {
+    if (this.miniHdfs != null) {
+      this.miniHdfs.shutdown();
+    }
+    super.tearDown();
+  }
+}