You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ya...@apache.org on 2020/12/09 06:36:37 UTC

[phoenix] branch 4.x updated: PHOENIX-6241 ViewIndexId sequences collide with ones created on the pre-4.15 version

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

yanxinyi pushed a commit to branch 4.x
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/4.x by this push:
     new 535a839  PHOENIX-6241 ViewIndexId sequences collide with ones created on the pre-4.15 version
535a839 is described below

commit 535a8391d60fac0d02777a47b6045f0ce294c00a
Author: Xinyi Yan <xy...@salesforce.com>
AuthorDate: Thu Dec 3 21:52:14 2020 -0800

    PHOENIX-6241 ViewIndexId sequences collide with ones created on the pre-4.15 version
    
    Signed-off-by: Xinyi Yan <ya...@apache.org>
---
 .../end2end/MergeViewIndexIdSequencesToolIT.java   |  93 ++++++++++++
 .../java/org/apache/phoenix/end2end/UpgradeIT.java |  11 +-
 .../phoenix/query/ConnectionQueryServicesImpl.java |  13 +-
 .../util/MergeViewIndexIdSequencesTool.java        | 117 ++++++++++++++
 .../java/org/apache/phoenix/util/UpgradeUtil.java  | 169 +++++++++++----------
 5 files changed, 317 insertions(+), 86 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MergeViewIndexIdSequencesToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MergeViewIndexIdSequencesToolIT.java
new file mode 100644
index 0000000..225437c
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MergeViewIndexIdSequencesToolIT.java
@@ -0,0 +1,93 @@
+/*
+ * 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 org.apache.phoenix.util.MergeViewIndexIdSequencesTool;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertTrue;
+
+@Category(NeedsOwnMiniClusterTest.class)
+public class MergeViewIndexIdSequencesToolIT extends ParallelStatsDisabledIT {
+    private final String CLEAN_QUERY = "DELETE FROM SYSTEM.\"SEQUENCE\"";
+    private final String COUNT_QUERY = "SELECT COUNT(*) FROM SYSTEM.\"SEQUENCE\"";
+    private final String UPSERT_QUERY = "UPSERT INTO SYSTEM.\"SEQUENCE\" " +
+            "(TENANT_ID, SEQUENCE_SCHEMA, SEQUENCE_NAME, START_WITH, CURRENT_VALUE, INCREMENT_BY," +
+            "CACHE_SIZE,MIN_VALUE,MAX_VALUE,CYCLE_FLAG,LIMIT_REACHED_FLAG) VALUES " +
+            "(?,?,?,?,?,?,?,?,?,?,?)";
+    private final String OLD_SEQUENCE_SCHEMA = "_SEQ_TEST.B";
+    private final String OLD_SEQUENCE_NAME = "_ID_";
+    private final String NEW_SEQUENCE_SCHEMA = "TEST";
+    private final String NEW_SEQUENCE_NAME = "B_ID_";
+
+    @Test
+    public void testOldSequenceFormat() throws Exception {
+        testSequenceRowCount(true);
+    }
+
+    @Test
+    public void testNewSequenceFormat() throws Exception {
+        testSequenceRowCount(false);
+    }
+
+    private void testSequenceRowCount(boolean isTestingOldFormat) throws Exception {
+        int expectedRowCount = isTestingOldFormat ? 2 : 1;
+        MergeViewIndexIdSequencesTool tool = new MergeViewIndexIdSequencesTool();
+        tool.setConf(config);
+        try(Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute(CLEAN_QUERY);
+            try (PreparedStatement preparedStatement = conn.prepareStatement(UPSERT_QUERY)) {
+                preparedStatement.setString(1, null);
+                if (isTestingOldFormat) {
+                    preparedStatement.setString(2, OLD_SEQUENCE_SCHEMA);
+                    preparedStatement.setString(3, OLD_SEQUENCE_NAME);
+                } else {
+                    preparedStatement.setString(2, NEW_SEQUENCE_SCHEMA);
+                    preparedStatement.setString(3, NEW_SEQUENCE_NAME);
+                }
+                preparedStatement.setLong(4, Short.MIN_VALUE);
+                preparedStatement.setLong(5, Short.MIN_VALUE + 1);
+                preparedStatement.setInt(6, 1);
+                preparedStatement.setInt(7, 1);
+                preparedStatement.setLong(8, Long.MIN_VALUE);
+                preparedStatement.setLong(9, Long.MAX_VALUE);
+                preparedStatement.setBoolean(10, false);
+                preparedStatement.setBoolean(11, false);
+                preparedStatement.execute();
+                conn.commit();
+            }
+            ResultSet rs = conn.createStatement().executeQuery(COUNT_QUERY);
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+
+            int status = tool.run(new String[]{"-r"});
+            assertEquals(0, status);
+
+            rs = conn.createStatement().executeQuery(COUNT_QUERY);
+            assertTrue(rs.next());
+            assertEquals(expectedRowCount, rs.getInt(1));
+        }
+    }
+}
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
index 8a6fac0..81b634d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
@@ -58,6 +58,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
@@ -386,7 +387,10 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
         cqs.clearCache();
         //Now make sure that running the merge logic doesn't cause a problem when there are no
         //sequences
-        UpgradeUtil.mergeViewIndexIdSequences(cqs, conn);
+        try (PhoenixConnection mockUpgradeScnTsConn = new PhoenixConnection(
+                conn, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0)) {
+            UpgradeUtil.mergeViewIndexIdSequences(mockUpgradeScnTsConn);
+        }
         PName tenantOne = PNameFactory.newName("TENANT_ONE");
         PName tenantTwo = PNameFactory.newName("TENANT_TWO");
         String tableName =
@@ -418,7 +422,10 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
             assertNull(e);
         }
 
-        UpgradeUtil.mergeViewIndexIdSequences(cqs, conn);
+        try (PhoenixConnection mockUpgradeScnTsConn = new PhoenixConnection(
+                conn, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0)) {
+            UpgradeUtil.mergeViewIndexIdSequences(mockUpgradeScnTsConn);
+        }
         //now check that there exists a sequence using the new naming convention, whose value is the
         //max of all the previous sequences for this table.
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 0bc4ad7..22227c2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -3901,9 +3901,19 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0) {
                     moveChildLinks = true;
                     syncAllTableAndIndexProps = true;
+                }
+                if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP_4_16_0) {
                     //Combine view index id sequences for the same physical view index table
                     //to avoid collisions. See PHOENIX-5132 and PHOENIX-5138
-                    UpgradeUtil.mergeViewIndexIdSequences(this, metaConnection);
+                    try (PhoenixConnection conn = new PhoenixConnection(
+                            ConnectionQueryServicesImpl.this, globalUrl,
+                            props, newEmptyMetaData())) {
+                        UpgradeUtil.mergeViewIndexIdSequences(metaConnection);
+                    } catch (Exception mergeViewIndeIdException) {
+                        LOGGER.warn("Merge view index id sequence failed! If possible, " +
+                                "please run MergeViewIndexIdSequencesTool to avoid view index" +
+                                "id collision. Error: " + mergeViewIndeIdException.getMessage());
+                    }
                 }
             }
 
@@ -3921,7 +3931,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             // create an entry for the SYSTEM namespace in the SYSCAT table, so that GRANT/REVOKE commands can work
             // with SYSTEM Namespace
             createSchemaIfNotExistsSystemNSMappingEnabled(metaConnection);
-
             clearUpgradeRequired();
             success = true;
         } catch (UpgradeInProgressException | UpgradeNotRequiredException e) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MergeViewIndexIdSequencesTool.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MergeViewIndexIdSequencesTool.java
new file mode 100644
index 0000000..cfd6d84
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MergeViewIndexIdSequencesTool.java
@@ -0,0 +1,117 @@
+/*
+ * 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.util;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.mapreduce.util.ConnectionUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MergeViewIndexIdSequencesTool extends Configured implements Tool {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(MergeViewIndexIdSequencesTool.class);
+
+
+    private static final Option HELP_OPTION = new Option("h", "help", false, "Help");
+    private static final Option RUN_OPTION = new Option("r", "run", false,
+            "Run MergeViewIndexIdSequencesTool to avoid view index id collision.");
+
+
+    private Options getOptions() {
+        final Options options = new Options();
+        options.addOption(RUN_OPTION);
+        options.addOption(HELP_OPTION);
+        return options;
+    }
+
+    private void parseOptions(String[] args) throws Exception {
+
+        final Options options = getOptions();
+
+        CommandLineParser parser = new PosixParser();
+        CommandLine cmdLine = null;
+        try {
+            cmdLine = parser.parse(options, args);
+        } catch (ParseException e) {
+            printHelpAndExit("Error parsing command line options: " + e.getMessage(), options);
+        }
+
+        if (cmdLine.hasOption(HELP_OPTION.getOpt())) {
+            printHelpAndExit(options, 0);
+        }
+
+        if (!cmdLine.hasOption(RUN_OPTION.getOpt())) {
+            printHelpAndExit("Please give at least one param", options);
+        }
+
+    }
+
+    private void printHelpAndExit(Options options, int exitCode) {
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.printHelp("help", options);
+        System.exit(exitCode);
+    }
+
+    private void printHelpAndExit(String errorMessage, Options options) {
+        System.err.println(errorMessage);
+        printHelpAndExit(options, 1);
+    }
+
+
+    @Override
+    public int run(String[] args) throws Exception {
+        int status = 0;
+        PhoenixConnection conn = null;
+        try {
+            parseOptions(args);
+
+            final Configuration config = HBaseConfiguration.addHbaseResources(getConf());
+
+            conn = ConnectionUtil.getInputConnection(config).
+                    unwrap(PhoenixConnection.class);
+
+            UpgradeUtil.mergeViewIndexIdSequences(conn);
+
+        } catch (Exception e) {
+            LOGGER.error("Get an error while running MergeViewIndexIdSequencesTool, "
+                    + e.getMessage());
+            status = 1;
+        } finally {
+            if (conn != null) {
+                conn.close();
+            }
+        }
+        return status;
+    }
+    public static void main(final String[] args) throws Exception {
+        int result = ToolRunner.run(new MergeViewIndexIdSequencesTool(), args);
+        System.exit(result);
+    }
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index 58c2757..22f058c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -2472,8 +2472,8 @@ public class UpgradeUtil {
         }
     }
 
-    public static void mergeViewIndexIdSequences(ConnectionQueryServices cqs, PhoenixConnection metaConnection)
-        throws SQLException{
+    public static void mergeViewIndexIdSequences(PhoenixConnection olderMetaConnection)
+            throws SQLException{
          /* before PHOENIX-5132, there was a per-tenant sequence to generate view index ids,
            which could cause problems if global and tenant-owned view indexes were mixed for the
            same physical base table. Now there's just one sequence for all view indexes of the same
@@ -2483,91 +2483,96 @@ public class UpgradeUtil {
          */
          //map of physical table names to view index sequences
         Map<String, List<SequenceKey>> sequenceTableMap = new HashMap<>();
-        DatabaseMetaData metaData = metaConnection.getMetaData();
-
-        try (ResultSet sequenceRS = metaData.getTables(null, null,
-            "%" + MetaDataUtil.VIEW_INDEX_SEQUENCE_NAME_PREFIX + "%",
-            new String[] {PhoenixDatabaseMetaData.SEQUENCE_TABLE_TYPE})) {
-            while (sequenceRS.next()) {
-                String tenantId = sequenceRS.getString(TABLE_CAT);
-                String schemaName = sequenceRS.getString(TABLE_SCHEM);
-                String sequenceName = sequenceRS.getString(TABLE_NAME);
-                int numBuckets = sequenceRS.getInt(SALT_BUCKETS);
-                SequenceKey key = new SequenceKey(tenantId, schemaName, sequenceName, numBuckets);
-                String baseTableName;
-                //under the old naming convention, view index sequences
-                // of non-namespace mapped tables stored their physical table name in the sequence schema for
-                //some reason. Namespace-mapped tables stored it in the sequence name itself.
-                //Note the difference between VIEW_INDEX_SEQUENCE_PREFIX (_SEQ_)
-                //and VIEW_INDEX_SEQUENCE_NAME_PREFIX (_ID_)
-                if (schemaName != null && schemaName.contains(MetaDataUtil.VIEW_INDEX_SEQUENCE_PREFIX)) {
-                    baseTableName = schemaName.replace(MetaDataUtil.VIEW_INDEX_SEQUENCE_PREFIX, "");
-                } else {
-                    baseTableName = SchemaUtil.getTableName(schemaName,
-                        sequenceName.replace(MetaDataUtil.VIEW_INDEX_SEQUENCE_NAME_PREFIX, ""));
-                }
-                if (!sequenceTableMap.containsKey(baseTableName)) {
-                    sequenceTableMap.put(baseTableName, new ArrayList<SequenceKey>());
+        try (PhoenixConnection metaConnection = new PhoenixConnection(
+                olderMetaConnection, HConstants.LATEST_TIMESTAMP)) {
+            DatabaseMetaData metaData = metaConnection.getMetaData();
+            ConnectionQueryServices cqs = metaConnection.getQueryServices();
+            try (ResultSet sequenceRS = metaData.getTables(null, null,
+                    "%" + MetaDataUtil.VIEW_INDEX_SEQUENCE_NAME_PREFIX + "%",
+                    new String[]{PhoenixDatabaseMetaData.SEQUENCE_TABLE_TYPE})) {
+                while (sequenceRS.next()) {
+                    String tenantId = sequenceRS.getString(TABLE_CAT);
+                    String schemaName = sequenceRS.getString(TABLE_SCHEM);
+                    String sequenceName = sequenceRS.getString(TABLE_NAME);
+                    int numBuckets = sequenceRS.getInt(SALT_BUCKETS);
+                    SequenceKey key = new SequenceKey(tenantId, schemaName, sequenceName, numBuckets);
+                    String baseTableName;
+                    //under the old naming convention, view index sequences
+                    // of non-namespace mapped tables stored their physical table name in the sequence schema for
+                    //some reason. Namespace-mapped tables stored it in the sequence name itself.
+                    //Note the difference between VIEW_INDEX_SEQUENCE_PREFIX (_SEQ_)
+                    //and VIEW_INDEX_SEQUENCE_NAME_PREFIX (_ID_)
+                    if (schemaName != null &&
+                            schemaName.contains(MetaDataUtil.VIEW_INDEX_SEQUENCE_PREFIX)) {
+                        baseTableName =
+                                schemaName.replace(MetaDataUtil.VIEW_INDEX_SEQUENCE_PREFIX, "");
+                    } else {
+                        baseTableName = SchemaUtil.getTableName(schemaName, sequenceName.
+                                replace(MetaDataUtil.VIEW_INDEX_SEQUENCE_NAME_PREFIX, ""));
+                    }
+                    if (!sequenceTableMap.containsKey(baseTableName)) {
+                        sequenceTableMap.put(baseTableName, new ArrayList<SequenceKey>());
+                    }
+                    sequenceTableMap.get(baseTableName).add(key);
                 }
-                sequenceTableMap.get(baseTableName).add(key);
             }
-        }
-        for (String baseTableName : sequenceTableMap.keySet()){
-            Map<SequenceKey, Long> currentSequenceValues = new HashMap<SequenceKey, Long>();
-            long maxViewIndexId = Long.MIN_VALUE;
-            PName name = PNameFactory.newName(baseTableName);
-            boolean hasNamespaceMapping =
-                SchemaUtil.isNamespaceMappingEnabled(null, cqs.getConfiguration()) ||
-                    cqs.getProps().getBoolean(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, false);
-            List<SequenceKey> existingSequenceKeys = sequenceTableMap.get(baseTableName);
-            for (SequenceKey sequenceKey : existingSequenceKeys){
-                long[] currentValueArray = new long[1];
-                SQLException[] sqlExceptions = new SQLException[1];
-                cqs.incrementSequences(
-                    Lists.newArrayList(new SequenceAllocation(sequenceKey, 1L)),
-                    EnvironmentEdgeManager.currentTimeMillis(),
-                    currentValueArray, new SQLException[1]);
-
-                if (sqlExceptions[0] != null) {
-                    LOGGER.error("Unable to convert view index sequence because of error. " +
-                        "It will need to be converted manually, " +
-                        " or there's a risk that two view indexes of the same base table " +
-                        "will have colliding view index ids.", sqlExceptions[0]);
-                    continue;
-                }
-                if (currentValueArray[0] > maxViewIndexId){
-                    maxViewIndexId = currentValueArray[0];
+            for (String baseTableName : sequenceTableMap.keySet()) {
+                Map<SequenceKey, Long> currentSequenceValues = new HashMap<SequenceKey, Long>();
+                long maxViewIndexId = Long.MIN_VALUE;
+                PName name = PNameFactory.newName(baseTableName);
+                boolean hasNamespaceMapping =
+                        SchemaUtil.isNamespaceMappingEnabled(null, cqs.getConfiguration()) ||
+                                cqs.getProps().getBoolean(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, false);
+                List<SequenceKey> existingSequenceKeys = sequenceTableMap.get(baseTableName);
+                for (SequenceKey sequenceKey : existingSequenceKeys) {
+                    long[] currentValueArray = new long[1];
+                    SQLException[] sqlExceptions = new SQLException[1];
+                    cqs.incrementSequences(
+                            Lists.newArrayList(new SequenceAllocation(sequenceKey, 1L)),
+                            EnvironmentEdgeManager.currentTimeMillis(),
+                            currentValueArray, new SQLException[1]);
+
+                    if (sqlExceptions[0] != null) {
+                        LOGGER.error("Unable to convert view index sequence because of error. " +
+                                "It will need to be converted manually, " +
+                                " or there's a risk that two view indexes of the same base table " +
+                                "will have colliding view index ids.", sqlExceptions[0]);
+                        continue;
+                    }
+                    if (currentValueArray[0] > maxViewIndexId) {
+                        maxViewIndexId = currentValueArray[0];
+                    }
+                    currentSequenceValues.put(sequenceKey, currentValueArray[0]);
                 }
-                currentSequenceValues.put(sequenceKey, currentValueArray[0]);
-            }
-            //just in case someone is creating a view index RIGHT NOW, increment maxViewIndexId
-            //by 100 to make very sure there are no collisions
-            maxViewIndexId += 100;
-            try {
-                //In one case (namespaced-mapped base table, global view index), the new sequence
-                //is the same as the old sequence, so rather than create it we just increment it
-                //to the right value.
-                SequenceKey newSequenceKey = new SequenceKey(null, MetaDataUtil.getViewIndexSequenceSchemaName(name, hasNamespaceMapping),
-                    MetaDataUtil.getViewIndexSequenceName(name, null, hasNamespaceMapping), cqs.getSequenceSaltBuckets());
-                if (currentSequenceValues.containsKey(newSequenceKey)){
-                    long incrementValue = maxViewIndexId - currentSequenceValues.get(newSequenceKey);
-                    SQLException[] incrementExceptions = new SQLException[1];
-                    List<SequenceAllocation> incrementAllocations = Lists.newArrayList(new SequenceAllocation(newSequenceKey, incrementValue));
-                    cqs.incrementSequences(incrementAllocations, EnvironmentEdgeManager.currentTimeMillis(),
-                        new long[1], incrementExceptions);
-                    if (incrementExceptions[0] != null){
-                        throw incrementExceptions[0];
+                //just in case someone is creating a view index RIGHT NOW, increment maxViewIndexId
+                //by 100 to make very sure there are no collisions
+                maxViewIndexId += 100;
+                try {
+                    //In one case (namespaced-mapped base table, global view index), the new sequence
+                    //is the same as the old sequence, so rather than create it we just increment it
+                    //to the right value.
+                    SequenceKey newSequenceKey = new SequenceKey(null, MetaDataUtil.getViewIndexSequenceSchemaName(name, hasNamespaceMapping),
+                            MetaDataUtil.getViewIndexSequenceName(name, null, hasNamespaceMapping), cqs.getSequenceSaltBuckets());
+                    if (currentSequenceValues.containsKey(newSequenceKey)) {
+                        long incrementValue = maxViewIndexId - currentSequenceValues.get(newSequenceKey);
+                        SQLException[] incrementExceptions = new SQLException[1];
+                        List<SequenceAllocation> incrementAllocations = Lists.newArrayList(new SequenceAllocation(newSequenceKey, incrementValue));
+                        cqs.incrementSequences(incrementAllocations, EnvironmentEdgeManager.currentTimeMillis(),
+                                new long[1], incrementExceptions);
+                        if (incrementExceptions[0] != null) {
+                            throw incrementExceptions[0];
+                        }
+                    } else {
+                        cqs.createSequence(null, newSequenceKey.getSchemaName(),
+                                newSequenceKey.getSequenceName(), maxViewIndexId, 1, 1,
+                                Long.MIN_VALUE, Long.MAX_VALUE,
+                                false, EnvironmentEdgeManager.currentTimeMillis());
                     }
-                } else {
-                    cqs.createSequence(null, newSequenceKey.getSchemaName(),
-                        newSequenceKey.getSequenceName(), maxViewIndexId, 1, 1,
-                        Long.MIN_VALUE, Long.MAX_VALUE,
-                        false, EnvironmentEdgeManager.currentTimeMillis());
+                } catch (SequenceAlreadyExistsException sae) {
+                    LOGGER.info("Tried to create view index sequence "
+                            + SchemaUtil.getTableName(sae.getSchemaName(), sae.getSequenceName()) +
+                            " during upgrade but it already existed. This is probably fine.");
                 }
-            } catch(SequenceAlreadyExistsException sae) {
-                LOGGER.info("Tried to create view index sequence "
-                    + SchemaUtil.getTableName(sae.getSchemaName(), sae.getSequenceName()) +
-                    " during upgrade but it already existed. This is probably fine.");
             }
         }
     }