You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/07/01 12:50:55 UTC

[GitHub] [hbase] virajjasani commented on a change in pull request #2009: HBASE-21596 Delete for a specific cell version can bring back version…

virajjasani commented on a change in pull request #2009:
URL: https://github.com/apache/hbase/pull/2009#discussion_r448324924



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
##########
@@ -3170,37 +3171,88 @@ public void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> f
           count = kvCount.get(qual);
 
           Get get = new Get(CellUtil.cloneRow(cell));
-          get.readVersions(count);
-          get.addColumn(family, qual);
+          get.readVersions(Integer.MAX_VALUE);
           if (coprocessorHost != null) {
             if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
                 byteNow, get)) {
-              updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
+              updateDeleteLatestVersionTimestamp(cell, get, count,
+                  this.htableDescriptor.getColumnFamily(family).getMaxVersions(),
+                    byteNow, deleteCells);
+
             }
           } else {
-            updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
+            updateDeleteLatestVersionTimestamp(cell, get, count,
+                this.htableDescriptor.getColumnFamily(family).getMaxVersions(),
+                  byteNow, deleteCells);
           }
         } else {
           PrivateCellUtil.updateLatestStamp(cell, byteNow);
+          deleteCells.add(cell);
         }
       }
+      e.setValue(deleteCells);
     }
   }
 
-  void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow)
-      throws IOException {
-    List<Cell> result = get(get, false);
-
+  void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, int maxVersions,
+      byte[] byteNow, List<Cell> deleteCells) throws IOException {
+    List<Cell> result = new ArrayList<>();
+    result.addAll(deleteCells);
+    Scan scan = new Scan(get);
+    scan.setRaw(true);
+    this.getScanner(scan).next(result);
+    result.sort((cell1, cell2) -> {
+      if(cell1.getTimestamp()>cell2.getTimestamp()){
+        return -1;
+      } else if(cell1.getTimestamp()<cell2.getTimestamp()){
+        return 1;
+      } else {
+        if(CellUtil.isDelete(cell1)){
+          return -1;
+        } else if (CellUtil.isDelete(cell2)){
+          return 1;
+        }
+      }
+      return 0;
+    });
+    List<Cell> cells = new ArrayList<>();
     if (result.size() < count) {

Review comment:
       result sorting doesn't seem useful for this condition. Can we avoid sorting for this?
   ```
   if (result.size() < count) {
   ..
   ..
     deleteCells.addAll(cells);
     return;
   }
   
   result.sort();
   
   if (result.size() > count){
   ..
   ..
   }else{
   ..
   }
   deleteCells.addAll(cells);
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
##########
@@ -3170,37 +3171,88 @@ public void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> f
           count = kvCount.get(qual);
 
           Get get = new Get(CellUtil.cloneRow(cell));
-          get.readVersions(count);
-          get.addColumn(family, qual);
+          get.readVersions(Integer.MAX_VALUE);
           if (coprocessorHost != null) {
             if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
                 byteNow, get)) {
-              updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
+              updateDeleteLatestVersionTimestamp(cell, get, count,
+                  this.htableDescriptor.getColumnFamily(family).getMaxVersions(),
+                    byteNow, deleteCells);
+
             }
           } else {
-            updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
+            updateDeleteLatestVersionTimestamp(cell, get, count,
+                this.htableDescriptor.getColumnFamily(family).getMaxVersions(),
+                  byteNow, deleteCells);
           }
         } else {
           PrivateCellUtil.updateLatestStamp(cell, byteNow);
+          deleteCells.add(cell);
         }
       }
+      e.setValue(deleteCells);
     }
   }
 
-  void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow)
-      throws IOException {
-    List<Cell> result = get(get, false);
-
+  void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, int maxVersions,

Review comment:
       nit: `private` ?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
##########
@@ -3170,37 +3171,88 @@ public void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> f
           count = kvCount.get(qual);
 
           Get get = new Get(CellUtil.cloneRow(cell));
-          get.readVersions(count);
-          get.addColumn(family, qual);
+          get.readVersions(Integer.MAX_VALUE);
           if (coprocessorHost != null) {
             if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
                 byteNow, get)) {
-              updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
+              updateDeleteLatestVersionTimestamp(cell, get, count,
+                  this.htableDescriptor.getColumnFamily(family).getMaxVersions(),
+                    byteNow, deleteCells);
+
             }
           } else {
-            updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
+            updateDeleteLatestVersionTimestamp(cell, get, count,
+                this.htableDescriptor.getColumnFamily(family).getMaxVersions(),
+                  byteNow, deleteCells);
           }
         } else {
           PrivateCellUtil.updateLatestStamp(cell, byteNow);
+          deleteCells.add(cell);
         }
       }
+      e.setValue(deleteCells);
     }
   }
 
-  void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow)
-      throws IOException {
-    List<Cell> result = get(get, false);
-
+  void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, int maxVersions,
+      byte[] byteNow, List<Cell> deleteCells) throws IOException {
+    List<Cell> result = new ArrayList<>();
+    result.addAll(deleteCells);

Review comment:
       nit: we can wrap it with ArrayList constructor: `new ArrayList<>(deleteCells);`

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
##########
@@ -3170,37 +3171,88 @@ public void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> f
           count = kvCount.get(qual);
 
           Get get = new Get(CellUtil.cloneRow(cell));
-          get.readVersions(count);
-          get.addColumn(family, qual);
+          get.readVersions(Integer.MAX_VALUE);
           if (coprocessorHost != null) {
             if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
                 byteNow, get)) {
-              updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
+              updateDeleteLatestVersionTimestamp(cell, get, count,
+                  this.htableDescriptor.getColumnFamily(family).getMaxVersions(),
+                    byteNow, deleteCells);
+
             }
           } else {
-            updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
+            updateDeleteLatestVersionTimestamp(cell, get, count,
+                this.htableDescriptor.getColumnFamily(family).getMaxVersions(),
+                  byteNow, deleteCells);
           }

Review comment:
       Would you prefer using a boolean to make single call to updateDeleteLatestVersionTimestamp?
   ```
             boolean updateDelTs=false;
             if (coprocessorHost != null) {
               if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
                   byteNow, get)) {
                 updateDelTs=true;
               }
             } else {
               updateDelTs=true;
             }
             if(updateDelTs){
               updateDeleteLatestVersionTimestamp(cell, get, count,
                 this.htableDescriptor.getColumnFamily(family).getMaxVersions(),
                 byteNow, deleteCells);
             }
   ```
   Only if you feel this is more readable :)

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
##########
@@ -3170,37 +3171,88 @@ public void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> f
           count = kvCount.get(qual);
 
           Get get = new Get(CellUtil.cloneRow(cell));
-          get.readVersions(count);
-          get.addColumn(family, qual);
+          get.readVersions(Integer.MAX_VALUE);
           if (coprocessorHost != null) {
             if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
                 byteNow, get)) {
-              updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
+              updateDeleteLatestVersionTimestamp(cell, get, count,
+                  this.htableDescriptor.getColumnFamily(family).getMaxVersions(),
+                    byteNow, deleteCells);
+
             }
           } else {
-            updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
+            updateDeleteLatestVersionTimestamp(cell, get, count,
+                this.htableDescriptor.getColumnFamily(family).getMaxVersions(),
+                  byteNow, deleteCells);
           }
         } else {
           PrivateCellUtil.updateLatestStamp(cell, byteNow);
+          deleteCells.add(cell);
         }
       }
+      e.setValue(deleteCells);
     }
   }
 
-  void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow)
-      throws IOException {
-    List<Cell> result = get(get, false);
-
+  void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, int maxVersions,
+      byte[] byteNow, List<Cell> deleteCells) throws IOException {
+    List<Cell> result = new ArrayList<>();
+    result.addAll(deleteCells);
+    Scan scan = new Scan(get);
+    scan.setRaw(true);
+    this.getScanner(scan).next(result);
+    result.sort((cell1, cell2) -> {
+      if(cell1.getTimestamp()>cell2.getTimestamp()){
+        return -1;
+      } else if(cell1.getTimestamp()<cell2.getTimestamp()){
+        return 1;
+      } else {
+        if(CellUtil.isDelete(cell1)){
+          return -1;
+        } else if (CellUtil.isDelete(cell2)){
+          return 1;
+        }
+      }
+      return 0;
+    });
+    List<Cell> cells = new ArrayList<>();
     if (result.size() < count) {
       // Nothing to delete
       PrivateCellUtil.updateLatestStamp(cell, byteNow);
-      return;
-    }
-    if (result.size() > count) {
-      throw new RuntimeException("Unexpected size: " + result.size());
+      cells.add(cell);
+      deleteCells.addAll(cells);
+    } else if (result.size() > count) {
+      int currentVersion = 0;
+      long latestCellTS = Long.MAX_VALUE;
+      for(Cell getCell : result){
+        if(!(CellUtil.matchingFamily(getCell, cell) && CellUtil.matchingQualifier(getCell, cell))){
+          continue;
+        }
+        if(!PrivateCellUtil.isDeleteType(getCell) && getCell.getTimestamp()!=latestCellTS){
+          if (currentVersion >= maxVersions) {
+            Cell tempCell = null;
+            try {
+              tempCell = PrivateCellUtil.deepClone(cell);
+            } catch (CloneNotSupportedException e) {
+              throw new IOException(e);
+            }
+            PrivateCellUtil.setTimestamp(tempCell, getCell.getTimestamp());
+            cells.add(tempCell);
+          } else if (currentVersion == 0) {
+            PrivateCellUtil.setTimestamp(cell, getCell.getTimestamp());
+            cells.add(cell);
+          }
+          currentVersion++;
+        }
+        latestCellTS = getCell.getTimestamp();
+      }
+
+    } else {
+      Cell getCell = result.get(0);
+      PrivateCellUtil.setTimestamp(cell, getCell.getTimestamp());
+      cells.add(cell);

Review comment:
       Should we not deepClone cell, update timestamp and then add that cloned cell to cells list similar to above branch?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org