You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2015/04/26 04:29:18 UTC

phoenix git commit: PHOENIX-1881 Set value in our empty key value to Tephra won't treat it like a column delete

Repository: phoenix
Updated Branches:
  refs/heads/txn 56cca1449 -> 3822f8669


PHOENIX-1881 Set value in our empty key value to Tephra won't treat it like a column delete


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

Branch: refs/heads/txn
Commit: 3822f8669755ea5c2faa4e31b3a0dc89cb7d5396
Parents: 56cca14
Author: Thomas <td...@salesforce.com>
Authored: Sat Apr 25 15:09:14 2015 -0700
Committer: Thomas <td...@salesforce.com>
Committed: Sat Apr 25 19:23:35 2015 -0700

----------------------------------------------------------------------
 .../StatsCollectorWithSplitsAndMultiCFIT.java       | 10 +++++-----
 .../apache/phoenix/transactions/TransactionIT.java  |  3 ---
 .../apache/phoenix/exception/SQLExceptionCode.java  |  2 +-
 .../org/apache/phoenix/execute/MutationState.java   | 12 ++++++------
 .../org/apache/phoenix/index/IndexMaintainer.java   |  3 ++-
 .../org/apache/phoenix/schema/MetaDataClient.java   | 16 +++++++---------
 .../java/org/apache/phoenix/schema/PTableImpl.java  |  4 +++-
 .../apache/phoenix/query/ConnectionlessTest.java    |  4 ++--
 8 files changed, 26 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3822f866/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorWithSplitsAndMultiCFIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorWithSplitsAndMultiCFIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorWithSplitsAndMultiCFIT.java
index 3c0c401..f8f7de0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorWithSplitsAndMultiCFIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorWithSplitsAndMultiCFIT.java
@@ -129,7 +129,7 @@ public class StatsCollectorWithSplitsAndMultiCFIT extends StatsCollectorAbstract
         Collection<GuidePostsInfo> infos = TestUtil.getGuidePostsList(conn, STATS_TEST_TABLE_NAME_NEW);
         long[] rowCountArr = new long[]{25, 20, 25, 25};
         // CF A alone has double the bytecount because it has column qualifier A and column qualifier _0
-        long[] byteCountArr = new long[]{12120, 5540, 6652, 6652};
+        long[] byteCountArr = new long[]{12168, 5540, 6652, 6652};
         int i = 0;
         for(GuidePostsInfo info : infos) {
             assertRowCountAndByteCount(info, rowCountArr[i], byteCountArr[i]);
@@ -138,8 +138,8 @@ public class StatsCollectorWithSplitsAndMultiCFIT extends StatsCollectorAbstract
     }
 
     protected void assertRowCountAndByteCount(GuidePostsInfo info, long rowCount, long byteCount) {
-        assertEquals(info.getRowCount(), rowCount);
-        assertEquals(info.getByteCount(), byteCount);
+        assertEquals("Row count does not match ", rowCount, info.getRowCount());
+        assertEquals("Byte count does not match ", byteCount, info.getByteCount());
     }
 
     @Test
@@ -166,7 +166,7 @@ public class StatsCollectorWithSplitsAndMultiCFIT extends StatsCollectorAbstract
         Collection<GuidePostsInfo> infos = TestUtil.getGuidePostsList(conn, STATS_TEST_TABLE_NAME);
         for (GuidePostsInfo info : infos) {
             assertEquals(20, info.getRowCount());
-            assertEquals(11020, info.getByteCount());
+            assertEquals(11060, info.getByteCount());
             break;
         }
         List<KeyRange> keyRanges = getAllSplits(conn, STATS_TEST_TABLE_NAME);
@@ -248,7 +248,7 @@ public class StatsCollectorWithSplitsAndMultiCFIT extends StatsCollectorAbstract
         infos = TestUtil.getGuidePostsList(conn, STATS_TEST_TABLE_NAME);
         for (GuidePostsInfo info : infos) {
             assertEquals(20, info.getRowCount());
-            assertEquals(9918, info.getByteCount());
+            assertEquals(9954, info.getByteCount());
             break;
         }
         conn.close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3822f866/phoenix-core/src/it/java/org/apache/phoenix/transactions/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/transactions/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/transactions/TransactionIT.java
index 240f3cd..a60ff32 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/transactions/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/transactions/TransactionIT.java
@@ -25,8 +25,6 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Map;
 
-import co.cask.tephra.TxConstants;
-
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -53,7 +51,6 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        props.put(TxConstants.ALLOW_EMPTY_VALUES_KEY, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
 	

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3822f866/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index 517b3ec..e3fbac8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -250,7 +250,7 @@ public enum SQLExceptionCode {
     DEFAULT_COLUMN_FAMILY_ON_SHARED_TABLE(1056, "43A13", "Default column family not allowed on VIEW or shared INDEX"),
     ONLY_TABLE_MAY_BE_DECLARED_TRANSACTIONAL(1070, "44A01", "Only tables may be declared as transactional"),
     MAY_NOT_MAP_TO_EXISTING_TABLE_AS_TRANSACTIONAL(1071, "44A02", "An existing HBase table may not be mapped to as a transactional table"),
-    STORE_NULLS_FOR_TRANSACTIONAL(1072, "44A03", "Store nulls must be true when a table is transactional"),
+    STORE_NULLS_MUST_BE_FALSE_FOR_TRANSACTIONAL(1072, "44A03", "Store nulls must be false when a table is transactional"),
 
     /** Sequence related */
     SEQUENCE_ALREADY_EXIST(1200, "42Z00", "Sequence already exists.", new Factory() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3822f866/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 0209e28..449e3cd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -422,21 +422,21 @@ public class MutationState implements SQLCloseable {
     }
     
     @SuppressWarnings("deprecation")
-    private void send(Iterator<TableRef> tableRefs) throws SQLException {
+    private void send(Iterator<TableRef> tableRefIterator) throws SQLException {
         int i = 0;
         long[] serverTimeStamps = null;
         byte[] tenantId = connection.getTenantId() == null ? null : connection.getTenantId().getBytes();
         // Validate up front if not transactional so that we 
-        if (tableRefs == null) {
+        if (tableRefIterator == null) {
             serverTimeStamps = validateAll();
-            tableRefs = mutations.keySet().iterator();
+            tableRefIterator = mutations.keySet().iterator();
         }
 
         // add tracing for this operation
         TraceScope trace = Tracing.startNewSpan(connection, "Committing mutations to tables");
         Span span = trace.getSpan();
-        while (tableRefs.hasNext()) {
-            TableRef tableRef = tableRefs.next();
+        while (tableRefIterator.hasNext()) {
+            TableRef tableRef = tableRefIterator.next();
             Map<ImmutableBytesPtr,Map<PColumn,byte[]>> valuesMap = mutations.get(tableRef);
             if (valuesMap == null) {
                 continue;
@@ -567,7 +567,7 @@ public class MutationState implements SQLCloseable {
             if (tableRef.getTable().getType() != PTableType.INDEX) {
                 numRows -= valuesMap.size();
             }
-            mutations.remove(tableRef); // Remove batches as we process them
+            tableRefIterator.remove(); // Remove batches as we process them
         }
         trace.close();
         assert(numRows==0);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3822f866/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 a2bd7a6..dbfca7b 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
@@ -760,7 +760,8 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             // add the keyvalue for the empty row
             put.add(kvBuilder.buildPut(new ImmutableBytesPtr(indexRowKey),
                 this.getEmptyKeyValueFamily(), QueryConstants.EMPTY_COLUMN_BYTES_PTR, ts,
-                ByteUtil.EMPTY_BYTE_ARRAY_PTR));
+                // set the vakue to the empty column name
+                QueryConstants.EMPTY_COLUMN_BYTES_PTR));
             put.setDurability(!indexWALDisabled ? Durability.USE_DEFAULT : Durability.SKIP_WAL);
         }
         int i = 0;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3822f866/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 7e332f9..f7d5af5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -1357,15 +1357,13 @@ public class MetaDataClient {
                     transactional = transactionalProp;
                 }
             }
-            if (transactional) { // FIXME: remove once Tephra handles column deletes
-                if (Boolean.FALSE.equals(storeNullsProp)) {
-                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.STORE_NULLS_FOR_TRANSACTIONAL)
-                    .setSchemaName(schemaName).setTableName(tableName)
-                    .build().buildException();
-                }
-                // Force STORE_NULLS to true when transactional as Tephra cannot deal with column deletes
-                storeNulls = true;
-                tableProps.put(PhoenixDatabaseMetaData.STORE_NULLS, Boolean.TRUE);
+            if (transactional) { // FIXME: remove once Tephra handles storing multiple versions of a cell value, 
+            	// and allows ignoring empty key values for an operation
+            	if (Boolean.FALSE.equals(storeNullsProp)) {
+            		throw new SQLExceptionInfo.Builder(SQLExceptionCode.STORE_NULLS_MUST_BE_FALSE_FOR_TRANSACTIONAL)
+            		.setSchemaName(schemaName).setTableName(tableName)
+            		.build().buildException();
+            	}
             }
 
             // Delay this check as it is supported to have IMMUTABLE_ROWS and SALT_BUCKETS defined on views

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3822f866/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 163daa2..4f2c796 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
@@ -653,7 +653,9 @@ public class PTableImpl implements PTable {
                 // way HBase works.
                 addQuietly(setValues, kvBuilder, kvBuilder.buildPut(keyPtr,
                     SchemaUtil.getEmptyColumnFamilyPtr(PTableImpl.this),
-                    QueryConstants.EMPTY_COLUMN_BYTES_PTR, ts, ByteUtil.EMPTY_BYTE_ARRAY_PTR));
+                    QueryConstants.EMPTY_COLUMN_BYTES_PTR, ts,
+                    // set the vakue to the empty column name
+                    QueryConstants.EMPTY_COLUMN_BYTES_PTR));
                 mutations.add(setValues);
                 if (!unsetValues.isEmpty()) {
                     mutations.add(unsetValues);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3822f866/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java
index 2b2841b..9b910d5 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java
@@ -152,7 +152,7 @@ public class ConnectionlessTest {
         assertTrue(iterator.hasNext());
         kv = iterator.next();
         assertArrayEquals(expectedRowKey1, kv.getRow());        
-        assertNull(PVarchar.INSTANCE.toObject(kv.getValue()));
+        assertEquals(QueryConstants.EMPTY_COLUMN_NAME, PVarchar.INSTANCE.toObject(kv.getValue()));
     }
 
     private static void assertRow2(Iterator<KeyValue> iterator, byte[] expectedRowKey2) {
@@ -168,7 +168,7 @@ public class ConnectionlessTest {
         assertTrue(iterator.hasNext());
         kv = iterator.next();
         assertArrayEquals(expectedRowKey2, kv.getRow());        
-        assertNull(PVarchar.INSTANCE.toObject(kv.getValue()));
+        assertEquals(QueryConstants.EMPTY_COLUMN_NAME, PVarchar.INSTANCE.toObject(kv.getValue()));
     }
     
     @Test