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 2015/04/30 08:51:20 UTC

[2/2] phoenix git commit: PHOENIX-1882 Issue column family deletes instead of row deletes in PTableImpl

PHOENIX-1882 Issue column family deletes instead of row deletes in PTableImpl


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

Branch: refs/heads/4.x-HBase-1.0
Commit: 6948e48f61401945501b36dc8194a980fbaf0564
Parents: 185fb78
Author: Thomas <td...@salesforce.com>
Authored: Sun Apr 26 11:38:51 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Wed Apr 29 23:48:46 2015 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/MappingTableDataTypeIT.java | 67 ++++++++++++++------
 .../apache/phoenix/index/IndexMaintainer.java   |  7 --
 .../org/apache/phoenix/schema/PTableImpl.java   |  8 +--
 3 files changed, 53 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6948e48f/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
index 98e536e..9617e37 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
@@ -19,23 +19,31 @@ package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
+import java.util.List;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
@@ -52,18 +60,44 @@ public class MappingTableDataTypeIT extends BaseHBaseManagedTimeIT {
         try {
             // Create table then get the single region for our new table.
             HTableDescriptor descriptor = new HTableDescriptor(tableName);
-            HColumnDescriptor columnDescriptor =  new HColumnDescriptor(Bytes.toBytes("cf"));
-            descriptor.addFamily(columnDescriptor);
+            HColumnDescriptor columnDescriptor1 =  new HColumnDescriptor(Bytes.toBytes("cf1"));
+            HColumnDescriptor columnDescriptor2 =  new HColumnDescriptor(Bytes.toBytes("cf2"));
+            descriptor.addFamily(columnDescriptor1);
+            descriptor.addFamily(columnDescriptor2);
             admin.createTable(descriptor);
             HTableInterface t = conn.getQueryServices().getTable(Bytes.toBytes("MTEST"));
             insertData(tableName.getName(), admin, t);
             t.close();
-            try {
-                testCreateTableMismatchedType();
-                fail();
-            } catch (SQLException e) {
-                assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(),e.getErrorCode());
-            }
+            // create phoenix table that maps to existing HBase table
+            createPhoenixTable();
+            
+            String selectSql = "SELECT * FROM MTEST";
+            ResultSet rs = conn.createStatement().executeQuery(selectSql);
+            ResultSetMetaData rsMetaData = rs.getMetaData();
+            assertTrue("Expected single row", rs.next());
+            // verify values from cf2 is not returned
+            assertEquals("Number of columns", 2, rsMetaData.getColumnCount());
+            assertEquals("Column Value", "value1", rs.getString(2));
+            assertFalse("Expected single row ", rs.next());
+            
+            // delete the row
+            String deleteSql = "DELETE FROM MTEST WHERE id = 'row'";
+            conn.createStatement().executeUpdate(deleteSql);
+            conn.commit();
+            
+            // verify that no rows are returned when querying through phoenix
+            rs = conn.createStatement().executeQuery(selectSql);
+            assertFalse("Expected no row` ", rs.next());
+            
+            // verify that row with value for cf2 still exists when using hbase apis
+            Scan scan = new Scan();
+            ResultScanner results = t.getScanner(scan);
+            Result result = results.next();
+            assertNotNull("Expected single row", result);
+            List<KeyValue> kvs = result.getColumn(Bytes.toBytes("cf2"), Bytes.toBytes("q2"));
+            assertEquals("Expected single value ", 1, kvs.size());
+            assertEquals("Column Value", "value2", Bytes.toString(kvs.get(0).getValue()));
+            assertNull("Expected single row", results.next());
         } finally {
             admin.close();
         }
@@ -72,26 +106,23 @@ public class MappingTableDataTypeIT extends BaseHBaseManagedTimeIT {
     private void insertData(final byte[] tableName, HBaseAdmin admin, HTableInterface t) throws IOException,
             InterruptedException {
         Put p = new Put(Bytes.toBytes("row"));
-        p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("value1"));
+        p.add(Bytes.toBytes("cf1"), Bytes.toBytes("q1"), Bytes.toBytes("value1"));
+        p.add(Bytes.toBytes("cf2"), Bytes.toBytes("q2"), Bytes.toBytes("value2"));
         t.put(p);
         t.flushCommits();
         admin.flush(tableName);
     }
 
     /**
-     * Test create a table in Phoenix with mismatched data type UNSIGNED_LONG
+     * Create a table in Phoenix that only maps column family cf1
      */
-    private void testCreateTableMismatchedType() throws SQLException {
+    private void createPhoenixTable() throws SQLException {
         String ddl = "create table IF NOT EXISTS MTEST (" + " id varchar NOT NULL primary key,"
-                + " \"cf\".\"q1\" unsigned_long" + " ) ";
+                + " \"cf1\".\"q1\" varchar" + " ) ";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(ddl);
         conn.commit();
-        String query = "select * from MTEST";
-        ResultSet rs = conn.createStatement().executeQuery(query);
-        rs.next();
-        rs.getLong(2);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6948e48f/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index 4565f39..0956753 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -806,15 +806,8 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         for (KeyValue kv : pendingUpdates) {
             if (kv.getTypeByte() == KeyValue.Type.DeleteFamily.getCode()) {
                 nDeleteCF++;
-                boolean isEmptyCF = Bytes.compareTo(kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), 
-                  dataEmptyKeyValueCF, 0, dataEmptyKeyValueCF.length) == 0;
-                // This is what a delete looks like on the client side for immutable indexing...
-                if (isEmptyCF) {
-                    return true;
-                }
             }
         }
-        // This is what a delete looks like on the server side for mutable indexing...
         return nDeleteCF == this.nDataCFs;
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6948e48f/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 2d523ff..edbefd5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -718,10 +718,10 @@ public class PTableImpl implements PTable {
         @Override
         public void delete() {
             newMutations();
-            // FIXME: the version of the Delete constructor without the lock args was introduced
-            // in 0.94.4, thus if we try to use it here we can no longer use the 0.94.2 version
-            // of the client.
-            Delete delete = new Delete(key,ts);
+            Delete delete = new Delete(key);
+            for (PColumnFamily colFamily : families) {
+            	delete.deleteFamily(colFamily.getName().getBytes(), ts);
+            }
             deleteRow = delete;
             // No need to write to the WAL for indexes
             if (PTableImpl.this.getType() == PTableType.INDEX) {