You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ka...@apache.org on 2019/09/12 08:52:05 UTC

[phoenix] branch PHOENIX-5473 created (now c04ecd2)

This is an automated email from the ASF dual-hosted git repository.

kadir pushed a change to branch PHOENIX-5473
in repository https://gitbox.apache.org/repos/asf/phoenix.git.


      at c04ecd2  PHOENIX-5473 Index write failures during index rebuilds should not change index table state

This branch includes the following new commits:

     new c04ecd2  PHOENIX-5473 Index write failures during index rebuilds should not change index table state

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[phoenix] 01/01: PHOENIX-5473 Index write failures during index rebuilds should not change index table state

Posted by ka...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kadir pushed a commit to branch PHOENIX-5473
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit c04ecd24c2f07b7c7d2657e1db9c233ed9cfc132
Author: Kadir <ko...@salesforce.com>
AuthorDate: Thu Sep 12 01:30:28 2019 -0700

    PHOENIX-5473 Index write failures during index rebuilds should not change index table state
---
 .../org/apache/phoenix/end2end/IndexToolIT.java    | 57 ++++++++++++++++++++++
 .../UngroupedAggregateRegionObserver.java          |  3 ++
 .../phoenix/index/PhoenixIndexFailurePolicy.java   |  7 ++-
 3 files changed, 65 insertions(+), 2 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
index 19d0e56..dc34013 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
@@ -54,6 +54,7 @@ import org.apache.phoenix.mapreduce.index.PhoenixServerBuildIndexMapper;
 
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.transaction.PhoenixTransactionProvider.Feature;
 import org.apache.phoenix.transaction.TransactionFactory;
@@ -334,6 +335,62 @@ public class IndexToolIT extends BaseUniqueNamesOwnClusterIT {
     }
 
     @Test
+    public void testSecondaryGlobalIndexFailure() throws Exception {
+        // Skip this test for local indexes and transactional tables
+        if (localIndex || transactional) {
+            return;
+        }
+        String schemaName = generateUniqueName();
+        String dataTableName = generateUniqueName();
+        String dataTableFullName = SchemaUtil.getTableName(schemaName, dataTableName);
+        String indexTableName = generateUniqueName();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            String stmString1 =
+                    "CREATE TABLE " + dataTableFullName
+                            + " (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, ZIP INTEGER) "
+                            + tableDDLOptions;
+            conn.createStatement().execute(stmString1);
+            String upsertQuery = String.format("UPSERT INTO %s VALUES(?, ?, ?)", dataTableFullName);
+            PreparedStatement stmt1 = conn.prepareStatement(upsertQuery);
+
+            // Insert two rows
+            upsertRow(stmt1, 1);
+            upsertRow(stmt1, 2);
+            conn.commit();
+
+            String stmtString2 =
+                    String.format(
+                            "CREATE %s INDEX %s ON %s  (LPAD(UPPER(NAME, 'en_US'),8,'x')||'_xyz') ASYNC ",
+                            (localIndex ? "LOCAL" : ""), indexTableName, dataTableFullName);
+            conn.createStatement().execute(stmtString2);
+
+            // Run the index MR job.
+            runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName);
+
+            String qIndexTableName = SchemaUtil.getQualifiedTableName(schemaName, indexTableName);
+
+            // Verify that the index table is in the ACTIVE state
+            assertEquals(PIndexState.ACTIVE, TestUtil.getIndexState(conn, qIndexTableName));
+
+            ConnectionQueryServices queryServices = conn.unwrap(PhoenixConnection.class).getQueryServices();
+            Admin admin = queryServices.getAdmin();
+            TableName tableName = TableName.valueOf(qIndexTableName);
+            admin.disableTable(tableName);
+
+            // Run the index MR job and it should fail (return -1)
+            runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+                    null, -1, new String[0]);
+
+            // Verify that the index table should be still in the ACTIVE state
+            assertEquals(PIndexState.ACTIVE, TestUtil.getIndexState(conn, qIndexTableName));
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
     public void testSaltedVariableLengthPK() throws Exception {
         if (!mutable) return;
         if (transactional) return;
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index 3d54dbc..3dadf7a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -260,6 +260,9 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         try {
             commitBatch(region, localRegionMutations, blockingMemstoreSize);
         } catch (IOException e) {
+            if (e instanceof DoNotRetryIOException) {
+                throw(e);
+            }
             handleIndexWriteException(localRegionMutations, e, new MutateCommand() {
                 @Override
                 public void doMutation() throws IOException {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
index 66e4250..6fdcc67 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
@@ -507,8 +507,11 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
                 throw new IOException(e);
             }
         }
-        // max retries hit - disable the index
-        handleIndexWriteFailureFromClient(iwe, connection);
+        if (!PhoenixIndexMetaData.isIndexRebuild(
+                mutateCommand.getMutationList().get(0).getAttributesMap())) {
+            // max retries hit - disable the index
+            handleIndexWriteFailureFromClient(iwe, connection);
+        }
         throw new DoNotRetryIOException(iwe); // send failure back to client
     }