You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by la...@apache.org on 2019/11/13 18:29:35 UTC

[phoenix] branch master updated: PHOENIX-5559 Fix remaining issues with Long viewIndexIds.

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

larsh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/master by this push:
     new 910b72b  PHOENIX-5559 Fix remaining issues with Long viewIndexIds.
910b72b is described below

commit 910b72bf5d3b51a0c30ce43d9b19c0ce089cda62
Author: Lars Hofhansl <la...@apache.org>
AuthorDate: Wed Nov 13 10:21:03 2019 -0800

    PHOENIX-5559 Fix remaining issues with Long viewIndexIds.
---
 .../end2end/BaseTenantSpecificViewIndexIT.java     |  10 +-
 .../org/apache/phoenix/end2end/BaseViewIT.java     |   8 +-
 .../phoenix/end2end/TenantSpecificViewIndexIT.java |   4 +-
 .../java/org/apache/phoenix/end2end/UpgradeIT.java |   2 +-
 .../it/java/org/apache/phoenix/end2end/ViewIT.java |  10 +-
 .../index/ChildViewsUseParentViewIndexIT.java      |   4 +-
 .../end2end/index/GlobalIndexOptimizationIT.java   |   2 +-
 .../apache/phoenix/end2end/index/IndexUsageIT.java |   4 +-
 .../apache/phoenix/end2end/index/LocalIndexIT.java |   2 +-
 .../end2end/index/MutableIndexFailureIT.java       |   2 +-
 .../phoenix/end2end/index/ShortViewIndexIdIT.java  | 104 +++++++++++++++++++++
 .../apache/phoenix/end2end/index/ViewIndexIT.java  |   4 +-
 .../coprocessor/BaseScannerRegionObserver.java     |  13 ++-
 .../phoenix/coprocessor/MetaDataEndpointImpl.java  |   6 +-
 .../org/apache/phoenix/iterate/ExplainTable.java   |   2 +-
 .../java/org/apache/phoenix/util/MetaDataUtil.java |  18 ++++
 16 files changed, 163 insertions(+), 32 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
index 216e2d3..9860624 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
@@ -140,18 +140,18 @@ public class BaseTenantSpecificViewIndexIT extends SplitSystemCatalogIT {
         ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT k1, k2, v2 FROM " + viewName + " WHERE v2='" + valuePrefix + "v2-1'");
         if(localIndex){
             assertEquals(saltBuckets == null ? 
-                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [" + Long.toString(1L + expectedIndexIdOffset) + ",'" + tenantId + "','" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [" + (1L + expectedIndexIdOffset) + ",'" + tenantId + "','" + valuePrefix + "v2-1']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY\n"
                             + "CLIENT MERGE SORT" :
-                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER " + tableName + " [" + Long.toString(1L + expectedIndexIdOffset) + ",'" + tenantId + "','" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER " + tableName + " [" + (1L + expectedIndexIdOffset) + ",'" + tenantId + "','" + valuePrefix + "v2-1']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY\n"
                             + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
         } else {
             String expected = saltBuckets == null ? 
-                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + tableName + " [" + Long.toString(Long.MIN_VALUE + expectedIndexIdOffset) + ",'" + tenantId + "','" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + tableName + " [" + (Short.MIN_VALUE + expectedIndexIdOffset) + ",'" + tenantId + "','" + valuePrefix + "v2-1']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY" :
-                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER _IDX_" + tableName + " [0," + Long.toString(Long.MIN_VALUE + expectedIndexIdOffset) + ",'" + tenantId + "','" + valuePrefix +
-                        "v2-1'] - ["+(saltBuckets.intValue()-1)+"," + Long.toString(Long.MIN_VALUE + expectedIndexIdOffset) + ",'" + tenantId + "','" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER _IDX_" + tableName + " [0," + (Short.MIN_VALUE + expectedIndexIdOffset) + ",'" + tenantId + "','" + valuePrefix +
+                        "v2-1'] - ["+(saltBuckets.intValue()-1)+"," + (Short.MIN_VALUE + expectedIndexIdOffset) + ",'" + tenantId + "','" + valuePrefix + "v2-1']\n"
 
                   + "    SERVER FILTER BY FIRST KEY ONLY\n"
                   + "CLIENT MERGE SORT";
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
index 8bfe6da..6a24cbd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
@@ -197,8 +197,8 @@ public abstract class BaseViewIT extends ParallelStatsEnabledIT {
                 queryPlan);
         } else {
             assertEquals(saltBuckets == null
-                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName +" [" + Long.MIN_VALUE + ",51]"
-                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [0," + Long.MIN_VALUE + ",51] - ["+(saltBuckets.intValue()-1)+"," + Long.MIN_VALUE + ",51]\nCLIENT MERGE SORT",
+                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName +" [" + Short.MIN_VALUE + ",51]"
+                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [0," + Short.MIN_VALUE + ",51] - ["+(saltBuckets.intValue()-1)+"," + Short.MIN_VALUE + ",51]\nCLIENT MERGE SORT",
                             queryPlan);
         }
 
@@ -238,9 +238,9 @@ public abstract class BaseViewIT extends ParallelStatsEnabledIT {
         } else {
             physicalTableName = viewIndexPhysicalName;
             assertEquals(saltBuckets == null
-                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName +" [" + (Long.MIN_VALUE+1) + ",'foo']\n"
+                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName +" [" + (Short.MIN_VALUE+1) + ",'foo']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY"
-                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [0," + (Long.MIN_VALUE+1) + ",'foo'] - ["+(saltBuckets.intValue()-1)+"," + (Long.MIN_VALUE+1) + ",'foo']\n"
+                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [0," + (Short.MIN_VALUE+1) + ",'foo'] - ["+(saltBuckets.intValue()-1)+"," + (Short.MIN_VALUE+1) + ",'foo']\n"
                                     + "    SERVER FILTER BY FIRST KEY ONLY\n"
                                     + "CLIENT MERGE SORT",
                             QueryUtil.getExplainPlan(rs));
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
index db2c1b0..37a31e3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
@@ -133,7 +133,7 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
         //are owned by different tenants.
         assertEquals(sequenceNameA, sequenceNameB);
         String sequenceSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(tableName), isNamespaceEnabled);
-        verifySequenceValue(null, sequenceNameA, sequenceSchemaName, Long.MIN_VALUE + 2L);
+        verifySequenceValue(null, sequenceNameA, sequenceSchemaName, Short.MIN_VALUE + 2L);
 
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId2);
@@ -209,7 +209,7 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
             assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER "
                     + Bytes.toString(MetaDataUtil.getViewIndexPhysicalName(
                         SchemaUtil.getPhysicalTableName(Bytes.toBytes(tableName), isNamespaceMapped).toBytes()))
-                    + " [" + Long.toString(Long.MIN_VALUE + indexIdOffset) + ",'" + tenantId + "','f']\n" + "    SERVER FILTER BY FIRST KEY ONLY",
+                    + " [" + Long.toString(Short.MIN_VALUE + indexIdOffset) + ",'" + tenantId + "','f']\n" + "    SERVER FILTER BY FIRST KEY ONLY",
                     QueryUtil.getExplainPlan(rs));
         }
 
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 facab6c..1f3d49e 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
@@ -188,7 +188,7 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
             PName physicalName = PNameFactory.newName(hbaseTableName);
             String newSchemaName = MetaDataUtil.getViewIndexSequenceSchemaName(physicalName, true);
             String newSequenceName = MetaDataUtil.getViewIndexSequenceName(physicalName, tenantId, true);
-            verifySequenceValue(null, newSequenceName, newSchemaName, -9223372036854775805L);
+            verifySequenceValue(null, newSequenceName, newSchemaName, Short.MIN_VALUE + 3);
             admin.close();
         }
     }
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index 1d1d20b..14e067b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -723,9 +723,9 @@ public class ViewIT extends SplitSystemCatalogIT {
                     queryPlan);
         } else {
             assertEquals(saltBuckets == null
-                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [" + Long.MIN_VALUE + ",51]"
+                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [" + Short.MIN_VALUE + ",51]"
                     : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [0,"
-                            + Long.MIN_VALUE + ",51] - [" + (saltBuckets.intValue() - 1) + "," + Long.MIN_VALUE
+                            + Short.MIN_VALUE + ",51] - [" + (saltBuckets.intValue() - 1) + "," + Short.MIN_VALUE
                             + ",51]\nCLIENT MERGE SORT",
                     queryPlan);
         }
@@ -767,10 +767,10 @@ public class ViewIT extends SplitSystemCatalogIT {
             assertEquals(
                     saltBuckets == null
                             ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " ["
-                                    + (Long.MIN_VALUE + 1) + ",'foo']\n" + "    SERVER FILTER BY FIRST KEY ONLY"
+                                    + (Short.MIN_VALUE + 1) + ",'foo']\n" + "    SERVER FILTER BY FIRST KEY ONLY"
                             : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName
-                                    + " [0," + (Long.MIN_VALUE + 1) + ",'foo'] - [" + (saltBuckets.intValue() - 1)
-                                    + "," + (Long.MIN_VALUE + 1) + ",'foo']\n"
+                                    + " [0," + (Short.MIN_VALUE + 1) + ",'foo'] - [" + (saltBuckets.intValue() - 1)
+                                    + "," + (Short.MIN_VALUE + 1) + ",'foo']\n"
                                     + "    SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT",
                     QueryUtil.getExplainPlan(rs));
         }
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
index 54528a9..8955653 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
@@ -161,7 +161,7 @@ public class ChildViewsUseParentViewIndexIT extends ParallelStatsDisabledIT {
         ResultSet rs = conn.prepareStatement("EXPLAIN " + sql).executeQuery();
         String childViewScanKey = isChildView ? ",'Y'" : "";
         assertEquals(
-            "CLIENT PARALLEL 1-WAY SKIP SCAN ON 3 KEYS OVER _IDX_" + baseTableName + " [-9223372036854775808,'1'" + childViewScanKey + "] - [-9223372036854775808,'3'" + childViewScanKey + "]\n" +
+            "CLIENT PARALLEL 1-WAY SKIP SCAN ON 3 KEYS OVER _IDX_" + baseTableName + " ["+ Short.MIN_VALUE +",'1'" + childViewScanKey + "] - [" + Short.MIN_VALUE + ",'3'" + childViewScanKey + "]\n" +
             "    SERVER FILTER BY FIRST KEY ONLY",
             QueryUtil.getExplainPlan(rs));
         
@@ -264,7 +264,7 @@ public class ChildViewsUseParentViewIndexIT extends ParallelStatsDisabledIT {
                 " ORDER BY WO_ID, A_DATE DESC";
         ResultSet rs = conn.prepareStatement("EXPLAIN " + sql).executeQuery();
         assertEquals(
-            "CLIENT PARALLEL 1-WAY SKIP SCAN ON 5 RANGES OVER _IDX_" + baseTableName + " [-9223372036854775808,'00Dxxxxxxxxxxx1','003xxxxxxxxxxx1',*] - [-9223372036854775808,'00Dxxxxxxxxxxx1','003xxxxxxxxxxx5',~'2016-01-01 06:00:00.000']\n" +
+            "CLIENT PARALLEL 1-WAY SKIP SCAN ON 5 RANGES OVER _IDX_" + baseTableName + " [" + Short.MIN_VALUE + ",'00Dxxxxxxxxxxx1','003xxxxxxxxxxx1',*] - [" + Short.MIN_VALUE + ",'00Dxxxxxxxxxxx1','003xxxxxxxxxxx5',~'2016-01-01 06:00:00.000']\n" +
             "    SERVER FILTER BY FIRST KEY ONLY",
             QueryUtil.getExplainPlan(rs));
         
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
index 09003b0..9567d36 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
@@ -344,7 +344,7 @@ public class GlobalIndexOptimizationIT extends ParallelStatsDisabledIT {
                     "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + dataTableName + "\n" +
                     "    SERVER FILTER BY V1 = 'a'\n" +
                     "    SKIP-SCAN-JOIN TABLE 0\n" +
-                    "        CLIENT PARALLEL 1-WAY SKIP SCAN ON 2 KEYS OVER _IDX_" + dataTableName + " \\[-9223372036854775808,1\\] - \\[-9223372036854775808,2\\]\n" +
+                    "        CLIENT PARALLEL 1-WAY SKIP SCAN ON 2 KEYS OVER _IDX_" + dataTableName + " \\[" + Short.MIN_VALUE + ",1\\] - \\[" + Short.MIN_VALUE + ",2\\]\n" +
                     "            SERVER FILTER BY FIRST KEY ONLY AND \"K2\" IN \\(3,4\\)\n" +
                     "    DYNAMIC SERVER FILTER BY \\(\"" + viewName + ".T_ID\", \"" + viewName + ".K1\", \"" + viewName + ".K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
             assertTrue("Expected:\n" + expected + "\ndid not match\n" + actual, Pattern.matches(expected,actual));
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexUsageIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexUsageIT.java
index 6433f5a..a3e68c2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexUsageIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexUsageIT.java
@@ -497,7 +497,7 @@ public class IndexUsageIT extends ParallelStatsDisabledIT {
 	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + dataTableName + " [1,173]\n" + "CLIENT MERGE SORT",
 	                    queryPlan);
 	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + dataTableName + " [" + Long.MIN_VALUE + ",173]", queryPlan);
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + dataTableName + " [" + Short.MIN_VALUE + ",173]", queryPlan);
 	        }
 	        rs = conn.createStatement().executeQuery(query);
 	        assertTrue(rs.next());
@@ -517,7 +517,7 @@ public class IndexUsageIT extends ParallelStatsDisabledIT {
 	                    + ",'foo2_bar2']\n" + "    SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT",
 	                    QueryUtil.getExplainPlan(rs));
 	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + dataTableName + " [" + (Long.MIN_VALUE + 1) + ",'foo2_bar2']\n"
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + dataTableName + " [" + (Short.MIN_VALUE + 1) + ",'foo2_bar2']\n"
 	                    + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
 	        }
 	        rs = conn.createStatement().executeQuery(query);
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
index 574322e..0be30c2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
@@ -429,7 +429,7 @@ public class LocalIndexIT extends BaseLocalIndexIT {
         Connection conn1 = getConnection();
         Connection conn2 = getConnection();
         conn1.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + tableName + "(v1)");
-        verifySequenceValue(null, sequenceName, sequenceSchemaName,-9223372036854775807L);
+        verifySequenceValue(null, sequenceName, sequenceSchemaName,Short.MIN_VALUE + 1);
         conn2.createStatement().executeQuery("SELECT * FROM " + tableName).next();
         conn1.createStatement().execute("DROP TABLE "+ tableName);
         verifySequenceNotExists(null, sequenceName, sequenceSchemaName);
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
index 334dd03..a5f72b8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
@@ -600,7 +600,7 @@ public class MutableIndexFailureIT extends BaseTest {
                             Cell firstCell = entry.getValue().get(0);
                             long indexId = MetaDataUtil.getViewIndexIdDataType().getCodec().decodeLong(firstCell.getRowArray(), firstCell.getRowOffset() + regionStartKeyLen, SortOrder.getDefault());
                             // Only throw for first local index as the test may have multiple local indexes
-                            if (indexId == Long.MIN_VALUE) {
+                            if (indexId == Short.MIN_VALUE) {
                                 throwException = true;
                                 break;
                             }
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ShortViewIndexIdIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ShortViewIndexIdIT.java
new file mode 100644
index 0000000..104504e
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ShortViewIndexIdIT.java
@@ -0,0 +1,104 @@
+/*
+ * 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.index;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.util.Map;
+
+import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class ShortViewIndexIdIT extends BaseUniqueNamesOwnClusterIT {
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
+        serverProps.put(QueryServices.LONG_VIEW_INDEX_ENABLED_ATTRIB, "false");
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+        clientProps.put(QueryServices.LONG_VIEW_INDEX_ENABLED_ATTRIB, "false");
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
+
+    @Test
+    public void testCreateLocalIndexWithData() throws Exception {
+        String tableName = generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement()
+                    .execute("CREATE TABLE " + tableName + " (pk INTEGER PRIMARY KEY, v INTEGER)");
+            conn.commit();
+            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES(1, 1)");
+            conn.commit();
+            conn.createStatement()
+                    .execute("CREATE LOCAL INDEX " + indexName + " ON " + tableName + "(v)");
+            conn.commit();
+            // this should return the expected data if the index was written correctly
+            ResultSet rs = conn.createStatement()
+                    .executeQuery("SELECT v FROM " + tableName + " WHERE v < 2");
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            rs.close();
+        }
+    }
+
+    @Test
+    public void testUpsertIntoLocalIndex() throws Exception {
+        String tableName = generateUniqueName();
+        String index1 = "IDX_" + generateUniqueName();
+        String index2 = "IDX_" + generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement()
+                    .execute("CREATE TABLE " + tableName + " (pk INTEGER PRIMARY KEY, v1 INTEGER, v2 INTEGER)");
+            conn.createStatement()
+            .execute("CREATE LOCAL INDEX " + index1 + " ON " + tableName + "(v1)");
+            conn.createStatement()
+            .execute("CREATE LOCAL INDEX " + index2 + " ON " + tableName + "(v2)");
+            conn.commit();
+            conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES(1, 2, 3)");
+            conn.commit();
+            ResultSet rs = conn.createStatement()
+                    .executeQuery("SELECT * FROM " + index1);
+            // we're expecting exactly one row mapping the column value to the key
+            assertTrue(rs.next());
+            assertEquals(2, rs.getInt(1));
+            assertEquals(1, rs.getInt(2));
+            assertFalse(rs.next());
+            rs.close();
+
+            rs = conn.createStatement()
+                    .executeQuery("SELECT * FROM " + index2);
+            // we're expecting exactly one row mapping the column value to the key
+            assertTrue(rs.next());
+            assertEquals(3, rs.getInt(1));
+            assertEquals(1, rs.getInt(2));
+            assertFalse(rs.next());
+            rs.close();
+        }
+    }
+
+}
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
index e3ff80d..6bb7ee3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
@@ -164,9 +164,9 @@ public class ViewIndexIT extends SplitSystemCatalogIT {
         conn2.createStatement().executeQuery("SELECT * FROM " + fullTableName).next();
         String sequenceName = getViewIndexSequenceName(PNameFactory.newName(fullTableName), null, isNamespaceMapped);
         String sequenceSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(fullTableName), isNamespaceMapped);
-        verifySequenceValue(null, sequenceName, sequenceSchemaName, Long.MIN_VALUE + 1);
+        verifySequenceValue(null, sequenceName, sequenceSchemaName, Short.MIN_VALUE + 1);
         conn1.createStatement().execute("CREATE INDEX " + indexName + "_2 ON " + fullViewName + " (v1)");
-        verifySequenceValue(null, sequenceName, sequenceSchemaName, Long.MIN_VALUE + 2);
+        verifySequenceValue(null, sequenceName, sequenceSchemaName, Short.MIN_VALUE + 2);
         conn1.createStatement().execute("DROP VIEW " + fullViewName);
         conn1.createStatement().execute("DROP TABLE "+ fullTableName);
         
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
index 1795c37..d393ed5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
@@ -162,11 +162,20 @@ abstract public class BaseScannerRegionObserver implements RegionObserver {
         byte[] upperExclusiveRegionKey = region.getRegionInfo().getEndKey();
         boolean isStaleRegionBoundaries;
         if (isLocalIndex) {
+            // For local indexes we have to abort any scan that was open during a split.
+            // We detect that condition as follows:
+            // 1. The scanner's stop row has to always match the region's end key.
+            // 2. Phoenix sets the SCAN_ACTUAL_START_ROW attribute to the scan's original start row
+            //    We cannot directly compare that with the region's start key, but can enforce that
+            //    the original start row still falls within the new region.
             byte[] expectedUpperRegionKey =
                     scan.getAttribute(EXPECTED_UPPER_REGION_KEY) == null ? scan.getStopRow() : scan
                             .getAttribute(EXPECTED_UPPER_REGION_KEY);
-            isStaleRegionBoundaries = expectedUpperRegionKey != null &&
-                    Bytes.compareTo(upperExclusiveRegionKey, expectedUpperRegionKey) != 0;
+
+            byte[] actualStartRow = scan.getAttribute(SCAN_ACTUAL_START_ROW);
+            isStaleRegionBoundaries = (expectedUpperRegionKey != null &&
+                    Bytes.compareTo(upperExclusiveRegionKey, expectedUpperRegionKey) != 0) || 
+                    (actualStartRow != null && Bytes.compareTo(actualStartRow, lowerInclusiveRegionKey) < 0);
         } else {
             isStaleRegionBoundaries = Bytes.compareTo(lowerInclusiveScanKey, lowerInclusiveRegionKey) < 0 ||
                     ( Bytes.compareTo(upperExclusiveScanKey, upperExclusiveRegionKey) > 0 && upperExclusiveRegionKey.length != 0) ||
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 c142e63..0be2383 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
@@ -1962,11 +1962,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
                         PName physicalName = parentTable.getPhysicalName();
                         long seqValue = getViewIndexSequenceValue(connection, tenantIdStr, parentTable, physicalName);
                         Put tableHeaderPut = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetadata);
-
                         NavigableMap<byte[], List<Cell>> familyCellMap = tableHeaderPut.getFamilyCellMap();
                         List<Cell> cells = familyCellMap.get(TABLE_FAMILY_BYTES);
                         Cell cell = cells.get(0);
-                        PDataType dataType = MetaDataUtil.getViewIndexIdDataType();
+                        PDataType<?> dataType = MetaDataUtil.getIndexDataType(tableMetadata,
+                                GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
                         Object val = dataType.toObject(seqValue, PLong.INSTANCE);
                         byte[] bytes = new byte[dataType.getByteSize() + 1];
                         dataType.toBytes(val, bytes, 0);
@@ -2103,7 +2103,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
         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);
+                    Short.MIN_VALUE, 1, 1, Long.MIN_VALUE, Long.MAX_VALUE, false, sequenceTimestamp);
         } catch (SequenceAlreadyExistsException e) {
             //someone else got here first and created the sequence, or it was pre-existing. Not a problem.
         }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
index e53b084..d8a0de9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
@@ -215,7 +215,7 @@ public abstract class ExplainTable {
     private Long getViewIndexValue(PDataType type, byte[] range) {
         boolean useLongViewIndex = MetaDataUtil.getViewIndexIdDataType().equals(type);
         Object s = type.toObject(range);
-        return (useLongViewIndex ? (Long) s : (Short) s) + (useLongViewIndex ? Long.MAX_VALUE : Short.MAX_VALUE) + 2;
+        return (useLongViewIndex ? (Long) s : (Short) s) + Short.MAX_VALUE + 2;
     }
 
     private static class RowKeyValueIterator implements Iterator<byte[]> {
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 3265e08..ec898f5 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
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.phoenix.coprocessor.MetaDataEndpointImpl;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
@@ -939,6 +940,23 @@ public class MetaDataUtil {
         return null;
     }
 
+	/**
+     * Retrieve the viewIndexId datatype from create request.
+     *
+     * @see MetaDataEndpointImpl#createTable(com.google.protobuf.RpcController,
+     *      org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest,
+     *      com.google.protobuf.RpcCallback)
+     */
+    public static PDataType<?> getIndexDataType(List<Mutation> tableMetaData,
+            KeyValueBuilder builder, ImmutableBytesWritable value) {
+        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData),
+                PhoenixDatabaseMetaData.VIEW_INDEX_ID_DATA_TYPE_BYTES, builder, value)) {
+            return PDataType.fromTypeId(
+                    PInteger.INSTANCE.getCodec().decodeInt(value, SortOrder.getDefault()));
+        }
+        return getLegacyViewIndexIdDataType();
+    }
+
     public static PColumn getColumn(int pkCount, byte[][] rowKeyMetaData, PTable table) throws ColumnFamilyNotFoundException, ColumnNotFoundException {
         PColumn col = null;
         if (pkCount > FAMILY_NAME_INDEX