You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2017/11/15 18:48:08 UTC

[13/37] phoenix git commit: PHOENIX-4277 Treat delete markers consistently with puts for point-in-time scans

PHOENIX-4277 Treat delete markers consistently with puts for point-in-time scans


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/7c21a83d
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/7c21a83d
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/7c21a83d

Branch: refs/heads/4.x-HBase-1.1
Commit: 7c21a83df97878f421464147d29a9dfd2d636870
Parents: 8b360e2
Author: James Taylor <jt...@salesforce.com>
Authored: Sun Oct 29 15:19:23 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Wed Nov 15 10:46:34 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/PointInTimeQueryIT.java     |  2 +-
 .../hadoop/hbase/regionserver/ScanInfoUtil.java | 35 ++++++++++++++++++++
 .../coprocessor/BaseScannerRegionObserver.java  | 21 ++++++++++++
 .../apache/phoenix/util/TransactionUtil.java    |  7 ++--
 4 files changed, 62 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7c21a83d/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
index c53e523..ed3e8a9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
@@ -63,7 +63,7 @@ public class PointInTimeQueryIT extends BaseQueryIT {
     public PointInTimeQueryIT(String idxDdl, boolean columnEncoded)
             throws Exception {
         // These queries fail without KEEP_DELETED_CELLS=true
-        super(idxDdl, columnEncoded, true);
+        super(idxDdl, columnEncoded, false);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7c21a83d/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfoUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfoUtil.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfoUtil.java
new file mode 100644
index 0000000..9d61437
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfoUtil.java
@@ -0,0 +1,35 @@
+/*
+ * 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 org.apache.hadoop.hbase.KeepDeletedCells;
+
+public class ScanInfoUtil {
+    private ScanInfoUtil() {
+    }
+    
+    public static boolean isKeepDeletedCells(ScanInfo scanInfo) {
+        return scanInfo.getKeepDeletedCells() != KeepDeletedCells.FALSE;
+    }
+    
+    public static ScanInfo cloneScanInfoWithKeepDeletedCells(ScanInfo scanInfo) {
+        return new ScanInfo(scanInfo.getFamily(), Math.max(scanInfo.getMinVersions(), 1),
+                    scanInfo.getMaxVersions(), scanInfo.getTtl(), KeepDeletedCells.TRUE,
+                    scanInfo.getTimeToPurgeDeletes(), scanInfo.getComparator());
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7c21a83d/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
index 1b95058..8aa9532 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
@@ -19,10 +19,12 @@ package org.apache.phoenix.coprocessor;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.NavigableSet;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
@@ -30,10 +32,15 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScanInfo;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.ScannerContextUtil;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
@@ -48,6 +55,7 @@ import org.apache.phoenix.schema.types.PUnsignedTinyint;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
+import org.apache.phoenix.util.TransactionUtil;
 
 abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
 
@@ -348,4 +356,17 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                 dataRegion, indexMaintainer, null, viewConstants, null, null, projector, ptr, useQualiferAsListIndex);
     }
 
+    @Override
+    public KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
+        final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
+        final KeyValueScanner s) throws IOException {
+
+      if (scan.isRaw() || ScanInfoUtil.isKeepDeletedCells(store.getScanInfo()) || scan.getTimeRange().getMax() == HConstants.LATEST_TIMESTAMP || TransactionUtil.isTransactionalTimestamp(scan.getTimeRange().getMax())) {
+        return s;
+      }
+
+      ScanInfo scanInfo = ScanInfoUtil.cloneScanInfoWithKeepDeletedCells(store.getScanInfo());
+      return new StoreScanner(store, scanInfo, scan, targetCols,
+          c.getEnvironment().getRegion().getReadpoint(scan.getIsolationLevel()));
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7c21a83d/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
index 01b775e..a99c700 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
@@ -24,8 +24,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
-import org.apache.phoenix.exception.SQLExceptionCode;
-import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.schema.PTable;
@@ -33,11 +31,16 @@ import org.apache.phoenix.transaction.PhoenixTransactionContext;
 import org.apache.phoenix.transaction.PhoenixTransactionalTable;
 import org.apache.phoenix.transaction.TephraTransactionTable;
 import org.apache.phoenix.transaction.TransactionFactory;
+import org.apache.tephra.util.TxUtils;
 
 public class TransactionUtil {
     private TransactionUtil() {
     }
     
+    public static boolean isTransactionalTimestamp(long ts) {
+        return !TxUtils.isPreExistingVersion(ts);
+    }
+    
     public static boolean isDelete(Cell cell) {
         return (CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY));
     }