Repository: phoenix
Updated Branches:
  refs/heads/master cdfb08bd8 -> 8c90a0bf4


PHOENIX-4162 Disallow transition from DISABLE to INACTIVE when 
INDEX_DISABLE_TIMESTAMP is zero


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

Branch: refs/heads/master
Commit: 8c90a0bf4269aa83bbed26afa3437e2d444c68bb
Parents: cdfb08b
Author: James Taylor <jtay...@salesforce.com>
Authored: Tue Sep 5 18:52:02 2017 -0700
Committer: James Taylor <jtay...@salesforce.com>
Committed: Wed Sep 6 00:25:27 2017 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/MetaDataEndPointIT.java     | 55 +++++++++++++++
 .../end2end/index/PartialIndexRebuilderIT.java  | 12 ++--
 .../coprocessor/MetaDataEndpointImpl.java       | 70 ++++++++++----------
 3 files changed, 98 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8c90a0bf/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndPointIT.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndPointIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndPointIT.java
new file mode 100644
index 0000000..08b8cc6
--- /dev/null
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndPointIT.java
@@ -0,0 +1,55 @@
+/*
+ * 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.phoenix.end2end;
+
+import static org.junit.Assert.assertEquals;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.Test;
+
+public class MetaDataEndPointIT extends ParallelStatsDisabledIT {
+    @Test
+       public void testUpdateIndexState() throws Throwable {
+        String schemaName = generateUniqueName();
+        String tableName = generateUniqueName();
+        String indexName1 = generateUniqueName();
+        final String fullTableName = SchemaUtil.getTableName(schemaName, 
tableName);
+        String fullIndexName1 = SchemaUtil.getTableName(schemaName, 
indexName1);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute("CREATE TABLE " + fullTableName + 
"(k INTEGER PRIMARY KEY, v1 INTEGER, v2 INTEGER) COLUMN_ENCODED_BYTES = 0, 
STORE_NULLS=true, GUIDE_POSTS_WIDTH=1000");
+            conn.createStatement().execute("CREATE INDEX " + indexName1 + " ON 
" + fullTableName + " (v1) INCLUDE (v2)");
+            conn.commit();
+            HTableInterface metaTable = 
conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            MutationCode code = IndexUtil.updateIndexState(fullIndexName1, 0L, 
metaTable, PIndexState.DISABLE).getMutationCode();
+            assertEquals(MutationCode.TABLE_ALREADY_EXISTS, code);
+            long ts = EnvironmentEdgeManager.currentTimeMillis();
+            code = IndexUtil.updateIndexState(fullIndexName1, ts, metaTable, 
PIndexState.INACTIVE).getMutationCode();
+            assertEquals(MutationCode.UNALLOWED_TABLE_MUTATION, code);
+        }
+       }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8c90a0bf/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
index 139725f..cacf0fa 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
@@ -258,23 +258,25 @@ public class PartialIndexRebuilderIT extends 
BaseUniqueNamesOwnClusterIT {
         EnvironmentEdgeManager.injectEdge(clock);
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement().execute("CREATE TABLE " + fullTableName + 
"(k INTEGER PRIMARY KEY, v1 INTEGER, v2 INTEGER) COLUMN_ENCODED_BYTES = 0, 
STORE_NULLS=true, GUIDE_POSTS_WIDTH=1000");
-            clock.time += 1000;
+            clock.time += 100;
             conn.createStatement().execute("CREATE INDEX " + indexName1 + " ON 
" + fullTableName + " (v1) INCLUDE (v2)");
-            clock.time += 1000;
+            clock.time += 100;
             conn.createStatement().execute("CREATE INDEX " + indexName2 + " ON 
" + fullTableName + " (v2) INCLUDE (v1)");
-            clock.time += 1000;
+            clock.time += 100;
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " 
VALUES(1, 2, 3)");
             conn.commit();
-            clock.time += 1000;
+            clock.time += 100;
             long disableTS = EnvironmentEdgeManager.currentTimeMillis();
             HTableInterface metaTable = 
conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName1, disableTS, metaTable, 
PIndexState.DISABLE);
             IndexUtil.updateIndexState(fullIndexName2, disableTS, metaTable, 
PIndexState.DISABLE);
+            clock.time += 100;
             TestUtil.doMajorCompaction(conn, fullIndexName1);
+            clock.time += 100;
             assertTrue(TestUtil.checkIndexState(conn, fullIndexName1, 
PIndexState.DISABLE, 0L));
-            TestUtil.analyzeTable(conn, fullTableName);
             assertFalse(TestUtil.checkIndexState(conn, fullIndexName2, 
PIndexState.DISABLE, 0L));
             TestUtil.doMajorCompaction(conn, fullTableName);
+            clock.time += 100;
             assertTrue(TestUtil.checkIndexState(conn, fullIndexName2, 
PIndexState.DISABLE, 0L));
         } finally {
             EnvironmentEdgeManager.injectEdge(null);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8c90a0bf/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index aac5619..5131a77 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -3484,41 +3484,42 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
                 PIndexState currentState =
                         
PIndexState.fromSerializedValue(currentStateKV.getValueArray()[currentStateKV
                                 .getValueOffset()]);
-
-                if ((currentDisableTimeStamp != null && 
currentDisableTimeStamp.getValueLength() > 0) &&
-                        (disableTimeStampKVIndex >= 0)) {
-                    long curTimeStampVal = (Long) 
PLong.INSTANCE.toObject(currentDisableTimeStamp.getValueArray(),
+                long curTimeStampVal = 0;
+                if ((currentDisableTimeStamp != null && 
currentDisableTimeStamp.getValueLength() > 0)) {
+                    curTimeStampVal = (Long) 
PLong.INSTANCE.toObject(currentDisableTimeStamp.getValueArray(),
                             currentDisableTimeStamp.getValueOffset(), 
currentDisableTimeStamp.getValueLength());
                     // new DisableTimeStamp is passed in
-                    Cell newDisableTimeStampCell = 
newKVs.get(disableTimeStampKVIndex);
-                    long newDisableTimeStamp = (Long) 
PLong.INSTANCE.toObject(newDisableTimeStampCell.getValueArray(),
-                            newDisableTimeStampCell.getValueOffset(), 
newDisableTimeStampCell.getValueLength());
-                    // We never set the INDEX_DISABLE_TIMESTAMP to a positive 
value when we're setting the state to ACTIVE.
-                    // Instead, we're passing in what we expect the 
INDEX_DISABLE_TIMESTAMP to be currently. If it's
-                    // changed, it means that a data table row failed to write 
while we were partially rebuilding it
-                    // and we must rerun it.
-                    if (newState == PIndexState.ACTIVE && newDisableTimeStamp 
> 0) {
-                        // Don't allow setting to ACTIVE if the 
INDEX_DISABLE_TIMESTAMP doesn't match
-                        // what we expect.
-                        if (newDisableTimeStamp != Math.abs(curTimeStampVal)) {
-                            
builder.setReturnCode(MetaDataProtos.MutationCode.UNALLOWED_TABLE_MUTATION);
-                            
builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                            done.run(builder.build());
-                            return;
-                       }
-                        // Reset INDEX_DISABLE_TIMESTAMP_BYTES to zero as 
we're good to go.
-                        newKVs.set(disableTimeStampKVIndex, 
-                                CellUtil.createCell(key, TABLE_FAMILY_BYTES, 
INDEX_DISABLE_TIMESTAMP_BYTES, 
-                                        timeStamp, 
KeyValue.Type.Put.getCode(), PLong.INSTANCE.toBytes(0L)));
-                    }
-                    // We use the sign of the INDEX_DISABLE_TIMESTAMP to 
differentiate the keep-index-active (negative)
-                    // from block-writes-to-data-table case. In either case, 
we want to keep the oldest timestamp to
-                    // drive the partial index rebuild rather than update it 
with each attempt to update the index
-                    // when a new data table write occurs.
-                    if (curTimeStampVal != 0 && Math.abs(curTimeStampVal) < 
Math.abs(newDisableTimeStamp)) {
-                        // not reset disable timestamp
-                        newKVs.remove(disableTimeStampKVIndex);
-                        disableTimeStampKVIndex = -1;
+                    if (disableTimeStampKVIndex >= 0) {
+                        Cell newDisableTimeStampCell = 
newKVs.get(disableTimeStampKVIndex);
+                        long newDisableTimeStamp = (Long) 
PLong.INSTANCE.toObject(newDisableTimeStampCell.getValueArray(),
+                                newDisableTimeStampCell.getValueOffset(), 
newDisableTimeStampCell.getValueLength());
+                        // We never set the INDEX_DISABLE_TIMESTAMP to a 
positive value when we're setting the state to ACTIVE.
+                        // Instead, we're passing in what we expect the 
INDEX_DISABLE_TIMESTAMP to be currently. If it's
+                        // changed, it means that a data table row failed to 
write while we were partially rebuilding it
+                        // and we must rerun it.
+                        if (newState == PIndexState.ACTIVE && 
newDisableTimeStamp > 0) {
+                            // Don't allow setting to ACTIVE if the 
INDEX_DISABLE_TIMESTAMP doesn't match
+                            // what we expect.
+                            if (newDisableTimeStamp != 
Math.abs(curTimeStampVal)) {
+                                
builder.setReturnCode(MetaDataProtos.MutationCode.UNALLOWED_TABLE_MUTATION);
+                                
builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                                done.run(builder.build());
+                                return;
+                           }
+                            // Reset INDEX_DISABLE_TIMESTAMP_BYTES to zero as 
we're good to go.
+                            newKVs.set(disableTimeStampKVIndex, 
+                                    CellUtil.createCell(key, 
TABLE_FAMILY_BYTES, INDEX_DISABLE_TIMESTAMP_BYTES, 
+                                            timeStamp, 
KeyValue.Type.Put.getCode(), PLong.INSTANCE.toBytes(0L)));
+                        }
+                        // We use the sign of the INDEX_DISABLE_TIMESTAMP to 
differentiate the keep-index-active (negative)
+                        // from block-writes-to-data-table case. In either 
case, we want to keep the oldest timestamp to
+                        // drive the partial index rebuild rather than update 
it with each attempt to update the index
+                        // when a new data table write occurs.
+                        if (curTimeStampVal != 0 && Math.abs(curTimeStampVal) 
< Math.abs(newDisableTimeStamp)) {
+                            // not reset disable timestamp
+                            newKVs.remove(disableTimeStampKVIndex);
+                            disableTimeStampKVIndex = -1;
+                        }
                     }
                 }
                 // Detect invalid transitions
@@ -3530,8 +3531,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
                         return;
                     }
                 } else if (currentState == PIndexState.DISABLE) {
+                    // Can't transition back to INACTIVE if 
INDEX_DISABLE_TIMESTAMP is 0
                     if (newState != PIndexState.BUILDING && newState != 
PIndexState.DISABLE &&
-                        newState != PIndexState.INACTIVE) {
+                        (newState != PIndexState.INACTIVE || curTimeStampVal 
== 0)) {
                         
builder.setReturnCode(MetaDataProtos.MutationCode.UNALLOWED_TABLE_MUTATION);
                         
builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
                         done.run(builder.build());

Reply via email to