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:34:55 UTC

[27/40] phoenix git commit: PHOENIX-4348 Point deletes do not work when there are immutable indexes with only row key columns

PHOENIX-4348 Point deletes do not work when there are immutable indexes with only row key columns


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 7062a48b48d8d59e1e53f4e6f7dae40b849b8064
Parents: cc604b7
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Nov 2 18:47:01 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Wed Nov 15 10:02:14 2017 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DeleteIT.java    | 96 +++++++++++++++++++-
 .../apache/phoenix/compile/DeleteCompiler.java  |  5 +-
 2 files changed, 94 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7062a48b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
index aa4d36e..9eac0af 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.end2end;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.sql.Connection;
@@ -32,10 +33,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
 
@@ -339,8 +337,6 @@ public class DeleteIT extends ParallelStatsDisabledIT {
                 con.commit();
             }
             
-            TestUtil.dumpTable(con.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName)));
-            
             ResultSet rs = con.createStatement().executeQuery("SELECT /*+ NO_INDEX */ count(*) FROM " + tableName);
             assertTrue(rs.next());
             assertEquals(0, rs.getLong(1));
@@ -370,6 +366,96 @@ public class DeleteIT extends ParallelStatsDisabledIT {
         testDeleteRowFromTableWithImmutableIndex(true, false);
     }
     
+    @Test
+    public void testPointDeleteRowFromTableWithImmutableIndex() throws Exception {
+        testPointDeleteRowFromTableWithImmutableIndex(false, false);
+    }
+    
+    @Test
+    public void testPointDeleteRowFromTableWithLocalImmutableIndex() throws Exception {
+        testPointDeleteRowFromTableWithImmutableIndex(true, false);
+    }
+    
+    @Test
+    public void testPointDeleteRowFromTableWithImmutableIndex2() throws Exception {
+        testPointDeleteRowFromTableWithImmutableIndex(false, true);
+    }
+    
+    public void testPointDeleteRowFromTableWithImmutableIndex(boolean localIndex, boolean addNonPKIndex) throws Exception {
+        Connection con = null;
+        try {
+            boolean autoCommit = false;
+            con = DriverManager.getConnection(getUrl());
+            con.setAutoCommit(autoCommit);
+
+            Statement stm = con.createStatement();
+
+            String tableName = generateUniqueName();
+            String indexName1 = generateUniqueName();
+            String indexName2 = generateUniqueName();
+            String indexName3 = addNonPKIndex? generateUniqueName() : null;
+
+            stm.execute("CREATE TABLE IF NOT EXISTS " + tableName + " (" +
+                    "HOST CHAR(2) NOT NULL," +
+                    "DOMAIN VARCHAR NOT NULL, " +
+                    "FEATURE VARCHAR NOT NULL, " +
+                    "\"DATE\" DATE NOT NULL, \n" + 
+                    "USAGE.CORE BIGINT," +
+                    "USAGE.DB BIGINT," +
+                    "STATS.ACTIVE_VISITOR INTEGER " +
+                    "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, \"DATE\")) IMMUTABLE_ROWS=true");
+            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (\"DATE\", FEATURE)");
+            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (FEATURE, DOMAIN)");
+            if (addNonPKIndex) {
+                stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName3 + " ON " + tableName + " (\"DATE\", FEATURE, USAGE.DB)");
+            }
+            
+            Date date = new Date(0);
+            PreparedStatement psInsert = con
+                    .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, \"DATE\", CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
+            psInsert.setString(1, "AA");
+            psInsert.setString(2, "BB");
+            psInsert.setString(3, "CC");
+            psInsert.setDate(4, date);
+            psInsert.setLong(5, 1L);
+            psInsert.setLong(6, 2L);
+            psInsert.setLong(7, 3);
+            psInsert.execute();
+            if (!autoCommit) {
+                con.commit();
+            }
+            
+            String dml = "DELETE FROM " + tableName + " WHERE (HOST, DOMAIN, FEATURE, \"DATE\") = (?,?,?,?)";
+            PreparedStatement psDelete = con.prepareStatement(dml);
+            psDelete.setString(1, "AA");
+            psDelete.setString(2, "BB");
+            psDelete.setString(3, "CC");
+            psDelete.setDate(4, date);
+            psDelete.execute();
+            if (!autoCommit) {
+                con.commit();
+            }
+            psDelete = con.prepareStatement("EXPLAIN " + dml);
+            psDelete.setString(1, "AA");
+            psDelete.setString(2, "BB");
+            psDelete.setString(3, "CC");
+            psDelete.setDate(4, date);
+            String explainPlan = QueryUtil.getExplainPlan(psDelete.executeQuery());
+            if (addNonPKIndex) {
+                assertNotEquals("DELETE SINGLE ROW", explainPlan);
+            } else {
+                assertEquals("DELETE SINGLE ROW", explainPlan);
+            }
+            
+            assertDeleted(con, tableName, indexName1, indexName2, indexName3);
+        } finally {
+            try {
+                con.close();
+            } catch (Exception ex) {
+            }
+        }
+    }
+        
     public void testDeleteRowFromTableWithImmutableIndex(boolean localIndex, boolean useCoveredIndex) throws Exception {
         Connection con = null;
         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7062a48b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 73689d5..f038cda 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -331,8 +331,9 @@ public class DeleteCompiler {
         @Override
         public MutationState execute() throws SQLException {
             MutationState state = firstPlan.execute();
+            statement.getConnection().getMutationState().join(state);
             for (MutationPlan plan : plans.subList(1, plans.size())) {
-                plan.execute();
+                statement.getConnection().getMutationState().join(plan.execute());
             }
             return state;
         }
@@ -564,7 +565,7 @@ public class DeleteCompiler {
                         while (iterator.hasNext()) {
                             mutation.put(new ImmutableBytesPtr(iterator.next().getLowerRange()), new RowMutationState(PRow.DELETE_MARKER, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
                         }
-                        return new MutationState(context.getCurrentTable(), mutation, 0, maxSize, maxSizeBytes, connection);
+                        return new MutationState(plan.getTableRef(), mutation, 0, maxSize, maxSizeBytes, connection);
                     }
     
                     @Override