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 2008/11/07 07:03:05 UTC

svn commit: r712068 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/client/ src/java/org/apache/hadoop/hbase/ipc/ src/java/org/apache/hadoop/hbase/regionserver/ src/java/org/apache/hadoop/hbase/regionserver/transactional/ src/test/org/apa...

Author: stack
Date: Thu Nov  6 22:02:59 2008
New Revision: 712068

URL: http://svn.apache.org/viewvc?rev=712068&view=rev
Log:
HBASE-961 Delete multiple columns by regular expression

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStore.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegion.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=712068&r1=712067&r2=712068&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Thu Nov  6 22:02:59 2008
@@ -101,6 +101,8 @@
    HBASE-81    When a scanner lease times out, throw a more "user friendly" exception
    HBASE-978   Remove BloomFilterDescriptor. It is no longer used.
    HBASE-975   Improve MapFile performance for start and end key
+   HBASE-961   Delete multiple columns by regular expression
+               (Samuel Guo via Stack)
         
   NEW FEATURES
    HBASE-875   Use MurmurHash instead of JenkinsHash [in bloomfilters]

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=712068&r1=712067&r2=712068&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 Thu Nov  6 22:02:59 2008
@@ -891,6 +891,71 @@
         }
     );
   }
+  
+  /** 
+   * Delete all cells that match the passed row and column.
+   * @param row Row to update
+   * @param colRegex column regex expression
+   * @throws IOException 
+   */
+  public void deleteAllByRegex(final String row, final String colRegex)
+  throws IOException {
+    deleteAll(row, colRegex, HConstants.LATEST_TIMESTAMP);
+  }
+
+  /** 
+   * Delete all cells that match the passed row and column and whose
+   * timestamp is equal-to or older than the passed timestamp.
+   * @param row Row to update
+   * @param colRegex Column Regex expression
+   * @param ts Delete all cells of the same timestamp or older.
+   * @throws IOException 
+   */
+  public void deleteAllByRegex(final String row, final String colRegex, 
+      final long ts) throws IOException {
+    deleteAllByRegex(Bytes.toBytes(row), colRegex, ts);
+  }
+
+  /** 
+   * Delete all cells that match the passed row and column and whose
+   * timestamp is equal-to or older than the passed timestamp.
+   * @param row Row to update
+   * @param colRegex Column Regex expression
+   * @param ts Delete all cells of the same timestamp or older.
+   * @throws IOException 
+   */
+  public void deleteAllByRegex(final byte [] row, final String colRegex, 
+      final long ts) throws IOException {
+    deleteAllByRegex(row, colRegex, ts, null);
+  }
+  
+  /** 
+   * Delete all cells that match the passed row and column and whose
+   * timestamp is equal-to or older than the passed timestamp, using an
+   * existing row lock.
+   * @param row Row to update
+   * @param colRegex Column regex expression
+   * @param ts Delete all cells of the same timestamp or older.
+   * @param rl Existing row lock
+   * @throws IOException 
+   */
+  public void deleteAllByRegex(final byte [] row, final String colRegex, 
+      final long ts, final RowLock rl)
+  throws IOException {
+    connection.getRegionServerWithRetries(
+        new ServerCallable<Boolean>(connection, tableName, row) {
+          public Boolean call() throws IOException {
+            long lockId = -1L;
+            if(rl != null) {
+              lockId = rl.getLockId();
+            }
+            this.server.deleteAllByRegex(location.getRegionInfo().getRegionName(),
+              row, colRegex, ts, lockId);
+            return null;
+          }
+        }
+    );
+  }
 
   /**
    * Delete all cells for a row with matching column family at all timestamps.
@@ -973,6 +1038,90 @@
         }
     );
   }
+  
+  /**
+   * Delete all cells for a row with matching column family regex 
+   * at all timestamps.
+   *
+   * @param row The row to operate on
+   * @param familyRegex Column family regex
+   * @throws IOException
+   */
+  public void deleteFamilyByRegex(final String row, final String familyRegex) 
+  throws IOException {
+    deleteFamilyByRegex(row, familyRegex, HConstants.LATEST_TIMESTAMP);
+  }
+
+  /**
+   * Delete all cells for a row with matching column family regex 
+   * at all timestamps.
+   *
+   * @param row The row to operate on
+   * @param familyRegex Column family regex
+   * @throws IOException
+   */
+  public void deleteFamilyByRegex(final byte[] row, final String familyRegex) 
+  throws IOException {
+    deleteFamilyByRegex(row, familyRegex, HConstants.LATEST_TIMESTAMP);
+  }
+
+  /**
+   * Delete all cells for a row with matching column family regex
+   * with timestamps less than or equal to <i>timestamp</i>.
+   *
+   * @param row The row to operate on
+   * @param familyRegex Column family regex
+   * @param timestamp Timestamp to match
+   * @throws IOException
+   */  
+  public void deleteFamilyByRegex(final String row, final String familyRegex,
+      final long timestamp)
+  throws IOException{
+    deleteFamilyByRegex(Bytes.toBytes(row), familyRegex, timestamp);
+  }
+
+  /**
+   * Delete all cells for a row with matching column family regex
+   * with timestamps less than or equal to <i>timestamp</i>.
+   *
+   * @param row The row to operate on
+   * @param familyRegex Column family regex
+   * @param timestamp Timestamp to match
+   * @throws IOException
+   */
+  public void deleteFamilyByRegex(final byte [] row, final String familyRegex, 
+    final long timestamp)
+  throws IOException {
+    deleteFamilyByRegex(row,familyRegex,timestamp,null);
+  }
+  
+  /**
+   * Delete all cells for a row with matching column family regex with
+   * timestamps less than or equal to <i>timestamp</i>, using existing
+   * row lock.
+   * 
+   * @param row The row to operate on
+   * @param familyRegex Column Family Regex
+   * @param timestamp Timestamp to match
+   * @param r1 Existing row lock
+   * @throws IOException
+   */
+  public void deleteFamilyByRegex(final byte[] row, final String familyRegex,
+    final long timestamp, final RowLock r1) throws IOException {
+    connection.getRegionServerWithRetries(
+        new ServerCallable<Boolean>(connection, tableName, row) {
+          public Boolean call() throws IOException {
+            long lockId = -1L;
+            if(r1 != null) {
+              lockId = r1.getLockId();
+            }
+            server.deleteFamilyByRegex(location.getRegionInfo().getRegionName(), 
+                row, familyRegex, timestamp, lockId);
+            return null;
+          }
+        }
+    );
+  }
 
   /**
    * Commit a BatchUpdate to the table.

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java?rev=712068&r1=712067&r2=712068&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java Thu Nov  6 22:02:59 2008
@@ -145,6 +145,21 @@
   public void deleteAll(byte [] regionName, byte [] row, long timestamp,
       long lockId)
   throws IOException;
+  
+  /**
+   * Delete all cells that match the passed row & the column regex and whose
+   * timestamp is equal-to or older than the passed timestamp.
+   * 
+   * @param regionName
+   * @param row
+   * @param colRegex
+   * @param timestamp
+   * @param lockId
+   * @throws IOException
+   */
+  public void deleteAllByRegex(byte [] regionName, byte [] row, String colRegex, 
+      long timestamp, long lockId)
+  throws IOException;
 
   /**
    * Delete all cells for a row with matching column family with timestamps
@@ -160,6 +175,21 @@
   public void deleteFamily(byte [] regionName, byte [] row, byte [] family, 
     long timestamp, long lockId)
   throws IOException;
+  
+  /**
+   * Delete all cells for a row with matching column family regex 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 familyRegex column family regex
+   * @param timestamp Timestamp to match
+   * @param lockId lock id
+   * @throws IOException
+   */
+  public void deleteFamilyByRegex(byte [] regionName, byte [] row, String familyRegex, 
+    long timestamp, long lockId) 
+  throws IOException;
 
   
   //

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=712068&r1=712067&r2=712068&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java Thu Nov  6 22:02:59 2008
@@ -36,6 +36,7 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -1299,7 +1300,7 @@
       if (targetStore != null) {
         // Pass versions without modification since in the store getKeys, it
         // includes the size of the passed <code>keys</code> array when counting.
-        List<HStoreKey> r = targetStore.getKeys(origin, versions, now);
+        List<HStoreKey> r = targetStore.getKeys(origin, versions, now, null);
         if (r != null) {
           keys.addAll(r);
         }
@@ -1533,7 +1534,7 @@
     try {
       for (HStore store : stores.values()) {
         List<HStoreKey> keys = store.getKeys(new HStoreKey(row, ts, this.regionInfo),
-          ALL_VERSIONS, now);
+          ALL_VERSIONS, now, null);
         TreeMap<HStoreKey, byte []> edits = new TreeMap<HStoreKey, byte []>(
           new HStoreKey.HStoreKeyWritableComparator(regionInfo));
         for (HStoreKey key: keys) {
@@ -1545,6 +1546,39 @@
       if(lockid == null) releaseRowLock(lid);
     }
   }
+  
+  /**
+   * Delete all cells for a row with matching columns with timestamps
+   * less than or equal to <i>timestamp</i>. 
+   * 
+   * @param row The row to operate on
+   * @param columnRegex The column regex 
+   * @param timestamp Timestamp to match
+   * @param lockid Row lock
+   * @throws IOException
+   */
+  @SuppressWarnings("unchecked")
+  public void deleteAllByRegex(final byte [] row, final String columnRegex, 
+      final long timestamp, final Integer lockid) throws IOException {
+    checkReadOnly();
+    Pattern columnPattern = Pattern.compile(columnRegex);
+    Integer lid = getLock(lockid, row);
+    long now = System.currentTimeMillis();
+    try {
+      for (HStore store : stores.values()) {
+        List<HStoreKey> keys = store.getKeys(new HStoreKey(row, timestamp, this.regionInfo),
+            ALL_VERSIONS, now, columnPattern);
+          TreeMap<HStoreKey, byte []> edits = new TreeMap<HStoreKey, byte []>(
+            new HStoreKey.HStoreKeyWritableComparator(regionInfo));
+          for (HStoreKey key: keys) {
+            edits.put(key, HLogEdit.deleteBytes.get());
+          }
+          update(edits);
+      }
+    } finally {
+      if(lockid == null) releaseRowLock(lid);
+    }
+  }
 
   /**
    * Delete all cells for a row with matching column family with timestamps
@@ -1568,7 +1602,7 @@
       HStore store = getStore(family);
       // find all the keys that match our criteria
       List<HStoreKey> keys = store.getKeys(new HStoreKey(row, timestamp,
-        this.regionInfo), ALL_VERSIONS, now);
+        this.regionInfo), ALL_VERSIONS, now, null);
       // delete all the cells
       TreeMap<HStoreKey, byte []> edits = new TreeMap<HStoreKey, byte []>(
         new HStoreKey.HStoreKeyWritableComparator(regionInfo));
@@ -1581,6 +1615,46 @@
     }
   }
   
+  /**
+   * Delete all cells for a row with all the matching column families by
+   * familyRegex with timestamps less than or equal to <i>timestamp</i>.
+   * 
+   * @param row The row to operate on
+   * @param familyRegex The column family regex for matching. This regex
+   * expression just match the family name, it didn't include <code>:<code>
+   * @param timestamp Timestamp to match
+   * @param lockid Row lock
+   * @throws IOException
+   */
+  @SuppressWarnings("unchecked")
+  public void deleteFamilyByRegex(byte [] row, String familyRegex, long timestamp, 
+      final Integer lockid) throws IOException {
+    checkReadOnly();
+    // construct the family regex pattern
+    Pattern familyPattern = Pattern.compile(familyRegex);
+    Integer lid = getLock(lockid, row);
+    long now = System.currentTimeMillis();
+    try {
+      for(HStore store : stores.values()) {
+        String familyName = Bytes.toString(store.getFamily().getName());
+        // check the family name match the family pattern.
+        if(!(familyPattern.matcher(familyName).matches())) 
+          continue;
+        
+        List<HStoreKey> keys = store.getKeys(new HStoreKey(row, timestamp, 
+            this.regionInfo), ALL_VERSIONS, now, null);
+        TreeMap<HStoreKey, byte []> edits = new TreeMap<HStoreKey, byte []>(
+          new HStoreKey.HStoreKeyWritableComparator(regionInfo));
+        for (HStoreKey key: keys) {
+          edits.put(key, HLogEdit.deleteBytes.get());
+        }
+        update(edits);
+      }
+    } finally {
+      if(lockid == null) releaseRowLock(lid);
+    }
+  }
+  
   /*
    * Delete one or many cells.
    * Used to support {@link #deleteAll(byte [], byte [], long)} and deletion of

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=712068&r1=712067&r2=712068&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Nov  6 22:02:59 2008
@@ -1388,6 +1388,13 @@
     region.deleteAll(row, timestamp, getLockFromId(lockId));
   }
 
+  @Override
+  public void deleteAllByRegex(byte[] regionName, byte[] row, String colRegex,
+      long timestamp, long lockId) throws IOException {
+    getRegion(regionName).deleteAllByRegex(row, colRegex, timestamp, 
+        getLockFromId(lockId));
+  }
+
   public void deleteFamily(byte [] regionName, byte [] row, byte [] family, 
     long timestamp, final long lockId)
   throws IOException{
@@ -1395,6 +1402,13 @@
         getLockFromId(lockId));
   }
 
+  @Override
+  public void deleteFamilyByRegex(byte[] regionName, byte[] row, String familyRegex,
+      long timestamp, long lockId) throws IOException {
+    getRegion(regionName).deleteFamilyByRegex(row, familyRegex, timestamp, 
+        getLockFromId(lockId));
+  }
+
   public long lockRow(byte [] regionName, byte [] row)
   throws IOException {
     checkOpen();

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStore.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStore.java?rev=712068&r1=712067&r2=712068&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStore.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStore.java Thu Nov  6 22:02:59 2008
@@ -1475,11 +1475,14 @@
    * @param versions How many versions to return. Pass
    * {@link HConstants#ALL_VERSIONS} to retrieve all.
    * @param now
+   * @param columnPattern regex pattern for column matching. if columnPattern
+   * is not null, we use column pattern to match columns. And the columnPattern
+   * only works when origin's column is null or its length is zero.
    * @return Matching keys.
    * @throws IOException
    */
   public List<HStoreKey> getKeys(final HStoreKey origin, final int versions,
-    final long now)
+    final long now, final Pattern columnPattern)
   throws IOException {
     // This code below is very close to the body of the get method.  Any 
     // changes in the flow below should also probably be done in get.  TODO:
@@ -1489,7 +1492,7 @@
     try {
       // Check the memcache
       List<HStoreKey> keys =
-        this.memcache.getKeys(origin, versions, deletes, now);
+        this.memcache.getKeys(origin, versions, deletes, now, columnPattern);
       // If we got sufficient versions from memcache, return. 
       if (keys.size() >= versions) {
         return keys;
@@ -1514,6 +1517,13 @@
           do {
             // if the row matches, we might want this one.
             if (rowMatches(origin, readkey)) {
+              // if the column pattern is not null, we use it for column matching.
+              // we will skip the keys whose column doesn't match the pattern.
+              if (columnPattern != null) {
+                if (!(columnPattern.matcher(Bytes.toString(readkey.getColumn())).matches())) {
+                  continue;
+                }
+              }
               // if the cell address matches, then we definitely want this key.
               if (cellMatches(origin, readkey)) {
                 // Store key if isn't deleted or superceded by memcache

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=712068&r1=712067&r2=712068&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 Nov  6 22:02:59 2008
@@ -33,6 +33,7 @@
 import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -601,21 +602,26 @@
    * {@link HConstants.ALL_VERSIONS} to retrieve all.
    * @param now
    * @param deletes Accumulating list of deletes
+   * @param columnPattern regex pattern for column matching. if columnPattern
+   * is not null, we use column pattern to match columns. And the columnPattern
+   * only works when origin's column is null or its length is zero.
    * @return Ordered list of <code>versions</code> keys going from newest back.
    * @throws IOException
    */
   List<HStoreKey> getKeys(final HStoreKey origin, final int versions,
-      final Set<HStoreKey> deletes, final long now) {
+      final Set<HStoreKey> deletes, final long now, 
+      final Pattern columnPattern) {
     this.lock.readLock().lock();
     try {
       List<HStoreKey> results;
       synchronized (memcache) {
-        results = getKeys(this.memcache, origin, versions, deletes, now);
+        results = 
+          getKeys(this.memcache, origin, versions, deletes, now, columnPattern);
       }
       synchronized (snapshot) {
         results.addAll(results.size(), getKeys(snapshot, origin,
             versions == HConstants.ALL_VERSIONS ? versions :
-              (versions - results.size()), deletes, now));
+              (versions - results.size()), deletes, now, columnPattern));
       }
       return results;
     } finally {
@@ -629,13 +635,17 @@
    * {@link HConstants.ALL_VERSIONS} to retrieve all.
    * @param now
    * @param deletes
+   * @param columnPattern regex pattern for column matching. if columnPattern
+   * is not null, we use column pattern to match columns. And the columnPattern
+   * only works when origin's column is null or its length is zero.
    * @return List of all keys that are of the same row and column and of
    * equal or older timestamp.  If no keys, returns an empty List. Does not
    * return null.
    */
   private List<HStoreKey> getKeys(final SortedMap<HStoreKey,
       byte []> map, final HStoreKey origin, final int versions,
-      final Set<HStoreKey> deletes, final long now) {
+      final Set<HStoreKey> deletes, final long now, 
+      final Pattern columnPattern) {
     List<HStoreKey> result = new ArrayList<HStoreKey>();
     List<HStoreKey> victims = new ArrayList<HStoreKey>();
     SortedMap<HStoreKey, byte []> tailMap = map.tailMap(origin);
@@ -649,6 +659,13 @@
             origin.getRow())) {
           break;
         }
+        // if the column pattern is not null, we use it for column matching.
+        // we will skip the keys whose column doesn't match the pattern.
+        if (columnPattern != null) {
+          if (!(columnPattern.matcher(Bytes.toString(key.getColumn())).matches())) {
+            continue;
+          }
+        }
         // if the rows match but the timestamp is newer, skip it so we can
         // get to the ones we actually want.
         if (key.getTimestamp() > origin.getTimestamp()) {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegion.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegion.java?rev=712068&r1=712067&r2=712068&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegion.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegion.java Thu Nov  6 22:02:59 2008
@@ -371,7 +371,7 @@
 
     for (HStore store : super.stores.values()) {
       List<HStoreKey> keys = store.getKeys(new HStoreKey(row, timestamp),
-          ALL_VERSIONS, now);
+          ALL_VERSIONS, now, null);
       BatchUpdate deleteUpdate = new BatchUpdate(row, timestamp);
 
       for (HStoreKey key : keys) {

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java?rev=712068&r1=712067&r2=712068&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java Thu Nov  6 22:02:59 2008
@@ -33,6 +33,9 @@
  */
 public class TestDeleteAll extends HBaseTestCase {
   static final Log LOG = LogFactory.getLog(TestDeleteAll.class);
+  
+  private final String COLUMN_REGEX = "[a-zA-Z0-9]*:[b|c]?";
+  
   private MiniDFSCluster miniHdfs;
   
   @Override
@@ -66,6 +69,11 @@
       // test hstore
       makeSureItWorks(region, region_incommon, true);
       
+      // regex test memcache
+      makeSureRegexWorks(region, region_incommon, false);
+      // regex test hstore
+      makeSureRegexWorks(region, region_incommon, true);
+      
     } finally {
       if (region != null) {
         try {
@@ -137,6 +145,79 @@
     
   }
   
+  private void makeSureRegexWorks(HRegion region, HRegionIncommon region_incommon, 
+      boolean flush)
+    throws Exception{
+      // insert a few versions worth of data for a row
+      byte [] row = Bytes.toBytes("test_row");
+      long t0 = System.currentTimeMillis();
+      long t1 = t0 - 15000;
+      long t2 = t1 - 15000;
+
+      byte [] colA = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a");
+      byte [] colB = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b");
+      byte [] colC = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "c");
+      byte [] colD = Bytes.toBytes(Bytes.toString(COLUMNS[0]));
+
+      BatchUpdate batchUpdate = new BatchUpdate(row, t0);
+      batchUpdate.put(colA, cellData(0, flush).getBytes());
+      batchUpdate.put(colB, cellData(0, flush).getBytes());
+      batchUpdate.put(colC, cellData(0, flush).getBytes());      
+      batchUpdate.put(colD, cellData(0, flush).getBytes());      
+      region_incommon.commit(batchUpdate);
+
+      batchUpdate = new BatchUpdate(row, t1);
+      batchUpdate.put(colA, cellData(1, flush).getBytes());
+      batchUpdate.put(colB, cellData(1, flush).getBytes());
+      batchUpdate.put(colC, cellData(1, flush).getBytes());      
+      batchUpdate.put(colD, cellData(1, flush).getBytes());      
+      region_incommon.commit(batchUpdate);
+      
+      batchUpdate = new BatchUpdate(row, t2);
+      batchUpdate.put(colA, cellData(2, flush).getBytes());
+      batchUpdate.put(colB, cellData(2, flush).getBytes());
+      batchUpdate.put(colC, cellData(2, flush).getBytes());      
+      batchUpdate.put(colD, cellData(2, flush).getBytes());      
+      region_incommon.commit(batchUpdate);
+
+      if (flush) {region_incommon.flushcache();}
+
+      // call delete the matching columns at a timestamp, 
+      // make sure only the most recent stuff is left behind
+      region.deleteAllByRegex(row, COLUMN_REGEX, t1, null);
+      if (flush) {region_incommon.flushcache();}    
+      assertCellEquals(region, row, colA, t0, cellData(0, flush));
+      assertCellEquals(region, row, colA, t1, cellData(1, flush));
+      assertCellEquals(region, row, colA, t2, cellData(2, flush));
+      assertCellEquals(region, row, colB, t0, cellData(0, flush));
+      assertCellEquals(region, row, colB, t1, null);
+      assertCellEquals(region, row, colB, t2, null);
+      assertCellEquals(region, row, colC, t0, cellData(0, flush));
+      assertCellEquals(region, row, colC, t1, null);
+      assertCellEquals(region, row, colC, t2, null);
+      assertCellEquals(region, row, colD, t0, cellData(0, flush));
+      assertCellEquals(region, row, colD, t1, null);
+      assertCellEquals(region, row, colD, t2, null);
+
+      // call delete all w/o a timestamp, make sure nothing is left.
+      region.deleteAllByRegex(row, COLUMN_REGEX, 
+          HConstants.LATEST_TIMESTAMP, null);
+      if (flush) {region_incommon.flushcache();}    
+      assertCellEquals(region, row, colA, t0, cellData(0, flush));
+      assertCellEquals(region, row, colA, t1, cellData(1, flush));
+      assertCellEquals(region, row, colA, t2, cellData(2, flush));
+      assertCellEquals(region, row, colB, t0, null);
+      assertCellEquals(region, row, colB, t1, null);
+      assertCellEquals(region, row, colB, t2, null);
+      assertCellEquals(region, row, colC, t0, null);
+      assertCellEquals(region, row, colC, t1, null);
+      assertCellEquals(region, row, colC, t2, null);
+      assertCellEquals(region, row, colD, t0, null);
+      assertCellEquals(region, row, colD, t1, null);
+      assertCellEquals(region, row, colD, t2, null);
+      
+    }
+  
   private String cellData(int tsNum, boolean flush){
     return "t" + tsNum + " data" + (flush ? " - with flush" : "");
   }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java?rev=712068&r1=712067&r2=712068&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java Thu Nov  6 22:02:59 2008
@@ -35,6 +35,9 @@
   static final Log LOG = LogFactory.getLog(TestDeleteFamily.class);
   private MiniDFSCluster miniHdfs;
 
+  //for family regex deletion test
+  protected static final String COLFAMILY_REGEX = "col[a-zA-Z]*1";
+  
   @Override
   protected void setUp() throws Exception {
     super.setUp();
@@ -60,6 +63,10 @@
       makeSureItWorks(region, region_incommon, false);
       // test hstore
       makeSureItWorks(region, region_incommon, true);
+      // family regex test memcache
+      makeSureRegexWorks(region, region_incommon, false);
+      // family regex test hstore
+      makeSureRegexWorks(region, region_incommon, true);
       
     } finally {
       if (region != null) {
@@ -138,6 +145,71 @@
     
   }
   
+  private void makeSureRegexWorks(HRegion region, HRegionIncommon region_incommon, 
+      boolean flush)
+    throws Exception{
+      // insert a few versions worth of data for a row
+      byte [] row = Bytes.toBytes("test_row");
+      long t0 = System.currentTimeMillis();
+      long t1 = t0 - 15000;
+      long t2 = t1 - 15000;
+
+      byte [] colA = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a");
+      byte [] colB = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b");
+      byte [] colC = Bytes.toBytes(Bytes.toString(COLUMNS[1]) + "c");
+
+      BatchUpdate batchUpdate = null;
+      batchUpdate = new BatchUpdate(row, t0);
+      batchUpdate.put(colA, cellData(0, flush).getBytes());
+      batchUpdate.put(colB, cellData(0, flush).getBytes());
+      batchUpdate.put(colC, cellData(0, flush).getBytes());      
+      region_incommon.commit(batchUpdate);
+
+      batchUpdate = new BatchUpdate(row, t1);
+      batchUpdate.put(colA, cellData(1, flush).getBytes());
+      batchUpdate.put(colB, cellData(1, flush).getBytes());
+      batchUpdate.put(colC, cellData(1, flush).getBytes());      
+      region_incommon.commit(batchUpdate);
+      
+      batchUpdate = new BatchUpdate(row, t2);
+      batchUpdate.put(colA, cellData(2, flush).getBytes());
+      batchUpdate.put(colB, cellData(2, flush).getBytes());
+      batchUpdate.put(colC, cellData(2, flush).getBytes());      
+      region_incommon.commit(batchUpdate);
+
+      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.deleteFamilyByRegex(row, COLFAMILY_REGEX, t1, null);
+      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
+      assertCellEquals(region, row, colA, t0, cellData(0, flush));
+      assertCellEquals(region, row, colA, t1, null);    
+      assertCellEquals(region, row, colA, t2, null);
+      assertCellEquals(region, row, colB, t0, cellData(0, flush));
+      assertCellEquals(region, row, colB, t1, null);
+      assertCellEquals(region, row, colB, t2, null);    
+      assertCellEquals(region, row, colC, t0, cellData(0, flush));
+      assertCellEquals(region, row, colC, t1, cellData(1, flush));
+      assertCellEquals(region, row, colC, t2, cellData(2, flush));        
+
+      // call delete family w/o a timestamp, make sure nothing is left except for
+      // column C.
+      region.deleteFamilyByRegex(row, COLFAMILY_REGEX, HConstants.LATEST_TIMESTAMP, null);
+      if (flush) {region_incommon.flushcache();}
+      // A,B for latest timestamp should be gone
+      // C should still be fine
+      assertCellEquals(region, row, colA, t0, null);
+      assertCellEquals(region, row, colB, t0, null);
+      assertCellEquals(region, row, colC, t0, cellData(0, flush));
+      assertCellEquals(region, row, colC, t1, cellData(1, flush));
+      assertCellEquals(region, row, colC, t2, cellData(2, flush));        
+      
+    }
+  
   private String cellData(int tsNum, boolean flush){
     return "t" + tsNum + " data" + (flush ? " - with flush" : "");
   }