You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by gj...@apache.org on 2019/03/26 18:21:55 UTC

[phoenix] branch PHOENIX-5138-2 updated (e357c6b -> 99d2f72)

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

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


 discard e357c6b  PHOENIX-5138 - ViewIndexId sequences created after PHOENIX-5132 shouldn't collide with ones created before it
     new 99d2f72  PHOENIX-5138 - ViewIndexId sequences created after PHOENIX-5132 shouldn't collide with ones created before it

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (e357c6b)
            \
             N -- N -- N   refs/heads/PHOENIX-5138-2 (99d2f72)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

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.


Summary of changes:
 phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java | 1 -
 1 file changed, 1 deletion(-)


[phoenix] 01/01: PHOENIX-5138 - ViewIndexId sequences created after PHOENIX-5132 shouldn't collide with ones created before it

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

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

commit 99d2f72c220a74b6bd157d2d40d400b3c7cd9586
Author: Geoffrey Jacoby <gj...@apache.org>
AuthorDate: Mon Mar 25 16:12:52 2019 -0700

    PHOENIX-5138 - ViewIndexId sequences created after PHOENIX-5132 shouldn't collide with ones created before it
---
 .../java/org/apache/phoenix/end2end/UpgradeIT.java | 99 +++++++++++++++++++++-
 .../phoenix/coprocessor/MetaDataEndpointImpl.java  | 47 +++++-----
 .../phoenix/query/ConnectionQueryServicesImpl.java |  6 ++
 .../java/org/apache/phoenix/util/MetaDataUtil.java | 30 ++++++-
 .../java/org/apache/phoenix/util/UpgradeUtil.java  | 98 +++++++++++++++++++++
 5 files changed, 256 insertions(+), 24 deletions(-)

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 632a2bb..76d84f6 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
@@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -30,6 +31,7 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.List;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.Callable;
@@ -38,6 +40,7 @@ import java.util.concurrent.FutureTask;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.collect.Lists;
 import org.apache.curator.shaded.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
@@ -60,6 +63,9 @@ import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.SequenceAllocation;
+import org.apache.phoenix.schema.SequenceKey;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -507,12 +513,20 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
         return DriverManager.getConnection(getUrl(), props);
     }
 
-    private Connection getConnection(boolean tenantSpecific, String tenantId) throws SQLException {
+    private Connection getConnection(boolean tenantSpecific, String tenantId, boolean isNamespaceMappingEnabled)
+        throws SQLException {
         if (tenantSpecific) {
             checkNotNull(tenantId);
             return createTenantConnection(tenantId);
         }
-        return DriverManager.getConnection(getUrl());
+        Properties props = new Properties();
+        if (isNamespaceMappingEnabled){
+            props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        }
+        return DriverManager.getConnection(getUrl(), props);
+    }
+    private Connection getConnection(boolean tenantSpecific, String tenantId) throws SQLException {
+        return getConnection(tenantSpecific, tenantId, false);
     }
     
     @Test
@@ -588,4 +602,85 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
         return childLinkSet;
     }
 
+    @Test
+    public void testMergeViewIndexSequences() throws Exception {
+        testMergeViewIndexSequencesHelper(false);
+    }
+
+    @Test
+    public void testMergeViewIndexSequencesWithNamespaces() throws Exception {
+        testMergeViewIndexSequencesHelper(true);
+    }
+
+    private void testMergeViewIndexSequencesHelper(boolean isNamespaceMappingEnabled) throws Exception {
+        PhoenixConnection conn = getConnection(false, null, isNamespaceMappingEnabled).unwrap(PhoenixConnection.class);
+        ConnectionQueryServices cqs = conn.getQueryServices();
+        //First delete any sequences that may exist from previous tests
+        conn.createStatement().execute("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE);
+        conn.commit();
+        cqs.clearCache();
+        //Now make sure that running the merge logic doesn't cause a problem when there are no
+        //sequences
+        UpgradeUtil.mergeViewIndexIdSequences(cqs, conn);
+        PName tenantOne = PNameFactory.newName("TENANT_ONE");
+        PName tenantTwo = PNameFactory.newName("TENANT_TWO");
+        String tableName =
+            SchemaUtil.getPhysicalHBaseTableName("TEST",
+                "T_" + generateUniqueName(), isNamespaceMappingEnabled).getString();
+        PName viewIndexTable = PNameFactory.newName(MetaDataUtil.getViewIndexPhysicalName(tableName));
+        SequenceKey sequenceOne =
+            createViewIndexSequenceWithOldName(cqs, tenantOne, viewIndexTable, isNamespaceMappingEnabled);
+        SequenceKey sequenceTwo =
+            createViewIndexSequenceWithOldName(cqs, tenantTwo, viewIndexTable, isNamespaceMappingEnabled);
+        SequenceKey sequenceGlobal =
+            createViewIndexSequenceWithOldName(cqs, null, viewIndexTable, isNamespaceMappingEnabled);
+
+        List<SequenceAllocation> allocations = Lists.newArrayList();
+        long val1 = 10;
+        long val2 = 100;
+        long val3 = 1000;
+        allocations.add(new SequenceAllocation(sequenceOne, val1));
+        allocations.add(new SequenceAllocation(sequenceGlobal, val2));
+        allocations.add(new SequenceAllocation(sequenceTwo, val3));
+
+
+        long[] incrementedValues = new long[3];
+        SQLException[] exceptions = new SQLException[3];
+        cqs.incrementSequences(allocations, EnvironmentEdgeManager.currentTimeMillis(), incrementedValues,
+            exceptions);
+        for (SQLException e : exceptions) {
+            assertNull(e);
+        }
+
+        UpgradeUtil.mergeViewIndexIdSequences(cqs, conn);
+        //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.
+
+        List<SequenceAllocation> afterUpgradeAllocations = Lists.newArrayList();
+        SequenceKey sequenceUpgrade = MetaDataUtil.getViewIndexSequenceKey(null, viewIndexTable, 0, isNamespaceMappingEnabled);
+        afterUpgradeAllocations.add(new SequenceAllocation(sequenceUpgrade, 1));
+        long[] afterUpgradeValues = new long[1];
+        SQLException[] afterUpgradeExceptions = new SQLException[1];
+        cqs.incrementSequences(afterUpgradeAllocations, EnvironmentEdgeManager.currentTimeMillis(), afterUpgradeValues, afterUpgradeExceptions);
+
+        assertNull(afterUpgradeExceptions[0]);
+        if (isNamespaceMappingEnabled){
+            //since one sequence (the global one) will be reused as the "new" sequence,
+            // it's already in cache and will reflect the final increment immediately
+            assertEquals(Long.MIN_VALUE + val3 + 1, afterUpgradeValues[0]);
+        } else {
+            assertEquals(Long.MIN_VALUE + val3, afterUpgradeValues[0]);
+        }
+    }
+
+    private SequenceKey createViewIndexSequenceWithOldName(ConnectionQueryServices cqs, PName tenant, PName viewIndexTable, boolean isNamespaceMapped) throws SQLException {
+        String tenantId = tenant == null ? null : tenant.getString();
+        SequenceKey key = MetaDataUtil.getOldViewIndexSequenceKey(tenantId, viewIndexTable, 0, isNamespaceMapped);
+        //Sequences are owned globally even if they contain a tenantId in the name
+        String sequenceTenantId = isNamespaceMapped ? tenantId : null;
+        cqs.createSequence(sequenceTenantId, key.getSchemaName(), key.getSequenceName(),
+            Long.MIN_VALUE, 1, 1, Long.MIN_VALUE, Long.MAX_VALUE, false, EnvironmentEdgeManager.currentTimeMillis());
+        return key;
+    }
+
 }
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 192d004..5e019bc 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
@@ -2362,26 +2362,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
                     String tenantIdStr = tenantIdBytes.length == 0 ? null : Bytes.toString(tenantIdBytes);
                     try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class)) {
                         PName physicalName = parentTable.getPhysicalName();
-                        int nSequenceSaltBuckets = connection.getQueryServices().getSequenceSaltBuckets();
-                        SequenceKey key = MetaDataUtil.getViewIndexSequenceKey(tenantIdStr, physicalName,
-                            nSequenceSaltBuckets, parentTable.isNamespaceMapped() );
-                        // TODO Review Earlier sequence was created at (SCN-1/LATEST_TIMESTAMP) and incremented at the client max(SCN,dataTable.getTimestamp), but it seems we should
-                        // use always LATEST_TIMESTAMP to avoid seeing wrong sequence values by different connection having SCN
-                        // or not.
-                        long sequenceTimestamp = HConstants.LATEST_TIMESTAMP;
-                        try {
-                            connection.getQueryServices().createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
-                                Long.MIN_VALUE, 1, 1, Long.MIN_VALUE, Long.MAX_VALUE, false, sequenceTimestamp);
-                        } catch (SequenceAlreadyExistsException e) {
-                        }
-                        long[] seqValues = new long[1];
-                        SQLException[] sqlExceptions = new SQLException[1];
-                        connection.getQueryServices().incrementSequences(Collections.singletonList(new SequenceAllocation(key, 1)),
-                            HConstants.LATEST_TIMESTAMP, seqValues, sqlExceptions);
-                        if (sqlExceptions[0] != null) {
-                            throw sqlExceptions[0];
-                        }
-                        long seqValue = seqValues[0];
+                        long seqValue = getViewIndexSequenceValue(connection, tenantIdStr, parentTable, physicalName);
                         Put tableHeaderPut = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetadata);
 
                         NavigableMap<byte[], List<Cell>> familyCellMap = tableHeaderPut.getFamilyCellMap();
@@ -2501,6 +2482,32 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
         }
     }
 
+    private long getViewIndexSequenceValue(PhoenixConnection connection, String tenantIdStr, PTable parentTable, PName physicalName) throws SQLException {
+        int nSequenceSaltBuckets = connection.getQueryServices().getSequenceSaltBuckets();
+
+        SequenceKey key = MetaDataUtil.getViewIndexSequenceKey(tenantIdStr, physicalName,
+            nSequenceSaltBuckets, parentTable.isNamespaceMapped() );
+        // Earlier sequence was created at (SCN-1/LATEST_TIMESTAMP) and incremented at the client max(SCN,dataTable.getTimestamp), but it seems we should
+        // use always LATEST_TIMESTAMP to avoid seeing wrong sequence values by different connection having SCN
+        // or not.
+        long sequenceTimestamp = HConstants.LATEST_TIMESTAMP;
+        try {
+            connection.getQueryServices().createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
+                Long.MIN_VALUE, 1, 1, Long.MIN_VALUE, Long.MAX_VALUE, false, sequenceTimestamp);
+        } catch (SequenceAlreadyExistsException e) {
+        }
+
+
+        long[] seqValues = new long[1];
+        SQLException[] sqlExceptions = new SQLException[1];
+        connection.getQueryServices().incrementSequences(Collections.singletonList(new SequenceAllocation(key, 1)),
+            HConstants.LATEST_TIMESTAMP, seqValues, sqlExceptions);
+        if (sqlExceptions[0] != null) {
+            throw sqlExceptions[0];
+        }
+        return seqValues[0];
+    }
+
     public static void dropChildViews(RegionCoprocessorEnvironment env, byte[] tenantIdBytes, byte[] schemaName, byte[] tableName)
             throws IOException, SQLException, ClassNotFoundException {
         Table hTable =
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 1f5cd48..ad9455f 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
@@ -70,7 +70,9 @@ import static org.apache.phoenix.util.UpgradeUtil.syncTableAndIndexProperties;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.ref.WeakReference;
+import java.sql.DatabaseMetaData;
 import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Types;
@@ -228,6 +230,7 @@ import org.apache.phoenix.schema.ReadOnlyTableException;
 import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.Sequence;
 import org.apache.phoenix.schema.SequenceAllocation;
+import org.apache.phoenix.schema.SequenceAlreadyExistsException;
 import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.SystemFunctionSplitPolicy;
@@ -3439,6 +3442,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // See PHOENIX-3955
                 if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0) {
                     syncTableAndIndexProperties(metaConnection, getAdmin());
+                    //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);
                 }
             }
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index 3c92a99..a3912cf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -618,13 +618,39 @@ public class MetaDataUtil {
         }
     }
 
-    public static String getViewIndexSequenceSchemaName(PName physicalName, boolean isNamespaceMapped) {
+    public static String getOldViewIndexSequenceSchemaName(PName physicalName, boolean isNamespaceMapped) {
         if (!isNamespaceMapped) { return VIEW_INDEX_SEQUENCE_PREFIX + physicalName.getString(); }
         return SchemaUtil.getSchemaNameFromFullName(physicalName.toString());
     }
 
+    public static String getOldViewIndexSequenceName(PName physicalName, PName tenantId, boolean isNamespaceMapped) {
+        if (!isNamespaceMapped) { return VIEW_INDEX_SEQUENCE_NAME_PREFIX + (tenantId == null ? "" : tenantId); }
+        return SchemaUtil.getTableNameFromFullName(physicalName.toString()) + VIEW_INDEX_SEQUENCE_NAME_PREFIX;
+    }
+
+    public static SequenceKey getOldViewIndexSequenceKey(String tenantId, PName physicalName, int nSaltBuckets,
+                                                      boolean isNamespaceMapped) {
+        // Create global sequence of the form: <prefixed base table name><tenant id>
+        // rather than tenant-specific sequence, as it makes it much easier
+        // to cleanup when the physical table is dropped, as we can delete
+        // all global sequences leading with <prefix> + physical name.
+        String schemaName = getOldViewIndexSequenceSchemaName(physicalName, isNamespaceMapped);
+        String tableName = getOldViewIndexSequenceName(physicalName, PNameFactory.newName(tenantId), isNamespaceMapped);
+        return new SequenceKey(isNamespaceMapped ? tenantId : null, schemaName, tableName, nSaltBuckets);
+    }
+
+    public static String getViewIndexSequenceSchemaName(PName physicalName, boolean isNamespaceMapped) {
+        if (!isNamespaceMapped) {
+            String baseTableName = SchemaUtil.getParentTableNameFromIndexTable(physicalName.getString(),
+                MetaDataUtil.VIEW_INDEX_TABLE_PREFIX);
+            return SchemaUtil.getSchemaNameFromFullName(baseTableName);
+        } else {
+            return SchemaUtil.getSchemaNameFromFullName(physicalName.toString());
+        }
+
+    }
+
     public static String getViewIndexSequenceName(PName physicalName, PName tenantId, boolean isNamespaceMapped) {
-        if (!isNamespaceMapped) { return VIEW_INDEX_SEQUENCE_NAME_PREFIX; }
         return SchemaUtil.getTableNameFromFullName(physicalName.toString()) + VIEW_INDEX_SEQUENCE_NAME_PREFIX;
     }
 
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 f0ee816..e4c4a5a 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
@@ -36,11 +36,13 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MAX_VALUE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MIN_VALUE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.START_WITH;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_CAT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM;
@@ -53,6 +55,7 @@ import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_
 
 import java.io.IOException;
 import java.sql.Connection;
+import java.sql.DatabaseMetaData;
 import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
@@ -117,6 +120,9 @@ import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SaltingUtil;
+import org.apache.phoenix.schema.SequenceAllocation;
+import org.apache.phoenix.schema.SequenceAlreadyExistsException;
+import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.types.PBinary;
@@ -2310,6 +2316,98 @@ public class UpgradeUtil {
         }
     }
 
+    public static void mergeViewIndexIdSequences(ConnectionQueryServices cqs, PhoenixConnection metaConnection)
+        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
+           physical table, but we have to check to see if there are any legacy sequences, and
+           merge them into a single sequence equal to max + 1 of the largest legacy sequence
+           to avoid collisons.
+         */
+        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>());
+                }
+                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) {
+                    continue;
+                }
+                if (currentValueArray[0] > maxViewIndexId){
+                    maxViewIndexId = currentValueArray[0];
+                }
+                currentSequenceValues.put(sequenceKey, currentValueArray[0]);
+            }
+            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());
+                }
+            } 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.");
+            }
+        }
+    }
+
     public static final String getSysCatalogSnapshotName(long currentSystemTableTimestamp) {
         String tableString = SYSTEM_CATALOG_NAME;
         Format formatter = new SimpleDateFormat("yyyyMMddHHmmss");