You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ra...@apache.org on 2016/05/24 15:29:03 UTC

[1/5] phoenix git commit: PHOENIX-1734 Local index improvements(Rajeshbabu)

Repository: phoenix
Updated Branches:
  refs/heads/master 18da4a046 -> 10909ae50


http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 34cedce..75b094b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -1114,7 +1114,7 @@ public class MetaDataClient {
         // connection so that our new index table is visible.
         Properties props = new Properties(connection.getClientInfo());
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(connection.getSCN()+1));
-        PhoenixConnection conn = DriverManager.getConnection(connection.getURL(), props).unwrap(PhoenixConnection.class);
+        PhoenixConnection conn = new PhoenixConnection(connection, connection.getQueryServices(), props);
         MetaDataClient newClientAtNextTimeStamp = new MetaDataClient(conn);
 
         // Re-resolve the tableRef from the now newer connection
@@ -1419,7 +1419,7 @@ public class MetaDataClient {
                     }
                     if (!SchemaUtil.isPKColumn(col) && col.getViewConstant() == null) {
                         // Need to re-create ColumnName, since the above one won't have the column family name
-                        colName = ColumnName.caseSensitiveColumnName(col.getFamilyName().getString(), IndexUtil.getIndexColumnName(col));
+                        colName = ColumnName.caseSensitiveColumnName(isLocalIndex?IndexUtil.getLocalIndexColumnFamily(col.getFamilyName().getString()):col.getFamilyName().getString(), IndexUtil.getIndexColumnName(col));
                         columnDefs.add(FACTORY.columnDef(colName, col.getDataType().getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, col.getSortOrder(), null, col.isRowTimestamp()));
                     }
                 }
@@ -1677,6 +1677,7 @@ public class MetaDataClient {
             boolean isNamespaceMapped = parent == null
                     ? SchemaUtil.isNamespaceMappingEnabled(tableType, connection.getQueryServices().getProps())
                     : parent.isNamespaceMapped();
+            boolean isLocalIndex = indexType == IndexType.LOCAL;
             if (parent != null && tableType == PTableType.INDEX) {
                 timestamp = TransactionUtil.getTableTimestamp(connection, transactional);
                 storeNulls = parent.getStoreNulls();
@@ -1686,17 +1687,21 @@ public class MetaDataClient {
                 // TODO: Can we support a multi-tenant index directly on a multi-tenant
                 // table instead of only a view? We don't have anywhere to put the link
                 // from the table to the index, though.
-                if (indexType == IndexType.LOCAL || (parent.getType() == PTableType.VIEW && parent.getViewType() != ViewType.MAPPED)) {
+                if (isLocalIndex || (parent.getType() == PTableType.VIEW && parent.getViewType() != ViewType.MAPPED)) {
                     PName physicalName = parent.getPhysicalName();
 
                     saltBucketNum = parent.getBucketNum();
-                    addSaltColumn = (saltBucketNum != null && indexType != IndexType.LOCAL);
+                    addSaltColumn = (saltBucketNum != null && !isLocalIndex);
                     defaultFamilyName = parent.getDefaultFamilyName() == null ? null : parent.getDefaultFamilyName().getString();
-                    if (indexType == IndexType.LOCAL) {
+                    if (isLocalIndex) {
+                        defaultFamilyName =
+                                parent.getDefaultFamilyName() == null ? QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY
+                                        : IndexUtil.getLocalIndexColumnFamily(parent.getDefaultFamilyName().getString());
                         saltBucketNum = null;
                         // Set physical name of local index table
-                        physicalNames = Collections.singletonList(PNameFactory.newName(MetaDataUtil.getLocalIndexPhysicalName(physicalName.getBytes())));
+                        physicalNames = Collections.singletonList(PNameFactory.newName(physicalName.getBytes()));
                     } else {
+                        defaultFamilyName = parent.getDefaultFamilyName() == null ? QueryConstants.DEFAULT_COLUMN_FAMILY : parent.getDefaultFamilyName().getString();
                         // Set physical name of view index table
                         physicalNames = Collections.singletonList(PNameFactory.newName(MetaDataUtil.getViewIndexPhysicalName(physicalName.getBytes())));
                     }
@@ -2050,7 +2055,9 @@ public class MetaDataClient {
                         .build().buildException();
                 }
                 if (column.getFamilyName() != null) {
-                    familyNames.put(column.getFamilyName().getString(),column.getFamilyName());
+                    familyNames.put(
+                        IndexUtil.getActualColumnFamilyName(column.getFamilyName().getString()),
+                        column.getFamilyName());
                 }
             }
             // We need a PK definition for a TABLE or mapped VIEW
@@ -2100,7 +2107,9 @@ public class MetaDataClient {
             throwIfInsufficientColumns(schemaName, tableName, pkColumns, saltBucketNum!=null, multiTenant);
 
             for (PName familyName : familyNames.values()) {
-                Collection<Pair<String,Object>> props = statement.getProps().get(familyName.getString());
+                String fam = familyName.getString();
+                Collection<Pair<String, Object>> props =
+                        statement.getProps().get(IndexUtil.getActualColumnFamilyName(fam));
                 if (props.isEmpty()) {
                     familyPropList.add(new Pair<byte[],Map<String,Object>>(familyName.getBytes(),commonFamilyProps));
                 } else {
@@ -2121,7 +2130,10 @@ public class MetaDataClient {
             if (familyNames.isEmpty()) {
                 //if there are no family names, use the default column family name. This also takes care of the case when
                 //the table ddl has only PK cols present (which means familyNames is empty).
-                byte[] cf = defaultFamilyName == null ? QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES : Bytes.toBytes(defaultFamilyName);
+                byte[] cf =
+                        defaultFamilyName == null ? (!isLocalIndex? QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES
+                                : QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES)
+                                : Bytes.toBytes(defaultFamilyName);
                 familyPropList.add(new Pair<byte[],Map<String,Object>>(cf, commonFamilyProps));
             }
 
@@ -2297,11 +2309,7 @@ public class MetaDataClient {
              */
             Collections.reverse(tableMetaData);
 
-            if (parent != null && tableType == PTableType.INDEX && indexType == IndexType.LOCAL) {
-                tableProps.put(MetaDataUtil.PARENT_TABLE_KEY, parent.getPhysicalName().getString());
-                tableProps.put(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_NAME, Boolean.TRUE);
-                splits = getSplitKeys(connection.getQueryServices().getAllTableRegions(parent.getPhysicalName().getBytes()));
-            } else {
+			if (indexType != IndexType.LOCAL) {
                 splits = SchemaUtil.processSplits(splits, pkColumns, saltBucketNum, connection.getQueryServices().getProps().getBoolean(
                         QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, QueryServicesOptions.DEFAULT_FORCE_ROW_KEY_ORDER));
             }
@@ -2566,7 +2574,7 @@ public class MetaDataClient {
                         // All multi-tenant tables have a view index table, so no need to check in that case
                         if (parentTableName == null) {
                             for (PTable index : table.getIndexes()) {
-                                if (MetaDataUtil.isLocalIndex(index.getPhysicalName().getString())) {
+                                if (index.getIndexType() == IndexType.LOCAL) {
                                     hasLocalIndexTable = true;
                                 } 
                             }
@@ -2590,19 +2598,6 @@ public class MetaDataClient {
                                         table.getColumnFamilies());
                                 tableRefs.add(new TableRef(null, viewIndexTable, ts, false));
                             }
-                            if (hasLocalIndexTable) {
-                                String localIndexSchemaName = null;
-                                String localIndexTableName = null;
-                                if (schemaName != null) {
-                                    localIndexSchemaName = MetaDataUtil.getLocalIndexTableName(schemaName);
-                                    localIndexTableName = tableName;
-                                } else {
-                                    localIndexTableName = MetaDataUtil.getLocalIndexTableName(tableName);
-                                }
-                                PTable localIndexTable = new PTableImpl(null, localIndexSchemaName, localIndexTableName,
-                                        ts, Collections.<PColumnFamily> emptyList());
-                                tableRefs.add(new TableRef(null, localIndexTable, ts, false));
-                            }
                         }
                         tableRefs.add(new TableRef(null, table, ts, false));
                         // TODO: Let the standard mutable secondary index maintenance handle this?
@@ -2639,6 +2634,17 @@ public class MetaDataClient {
                 buf.append("'" + ref.getTable().getPhysicalName().getString() + "',");
             }
             buf.setCharAt(buf.length() - 1, ')');
+            if(tableRefs.get(0).getTable().getIndexType()==IndexType.LOCAL) {
+                buf.append(" AND COLUMN_FAMILY IN(");
+                               if (tableRefs.get(0).getTable().getColumnFamilies().isEmpty()) {
+                    buf.append("'" + QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY + "',");
+                } else {
+                    for(PColumnFamily cf : tableRefs.get(0).getTable().getColumnFamilies()) {
+                        buf.append("'" + cf.getName().getString() + "',");
+                    }
+                }
+                   buf.setCharAt(buf.length() - 1, ')');
+            }
             conn.createStatement().execute(buf.toString());
             success = true;
         } catch (SQLException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
index e231342..2ce5160 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
@@ -21,6 +21,7 @@ import java.util.Objects;
 
 import org.apache.phoenix.compile.TupleProjectionCompiler;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -101,7 +102,7 @@ public class TableRef {
             String defaultFamilyName = table.getDefaultFamilyName() == null ? QueryConstants.DEFAULT_COLUMN_FAMILY : table.getDefaultFamilyName().getString();
             // Translate to the data table column name
             String dataFamilyName = isIndex ? IndexUtil.getDataColumnFamilyName(name) : column.getFamilyName().getString() ;
-            cf = defaultFamilyName.equals(dataFamilyName) ? null : dataFamilyName;
+            cf = (table.getIndexType()==IndexType.LOCAL? IndexUtil.getActualColumnFamilyName(defaultFamilyName):defaultFamilyName).equals(dataFamilyName) ? null : dataFamilyName;
             cq = isIndex ? IndexUtil.getDataColumnName(name) : name;
         }
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index 1f87774..5532d71 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.util;
 
+import static org.apache.phoenix.query.QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX;
 import static org.apache.phoenix.util.PhoenixRuntime.getTable;
 
 import java.io.ByteArrayInputStream;
@@ -24,6 +25,7 @@ import java.io.DataInputStream;
 import java.io.IOException;
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -40,10 +42,8 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.compile.ColumnResolver;
@@ -141,24 +141,37 @@ public class IndexUtil {
         return name.substring(0,name.indexOf(INDEX_COLUMN_NAME_SEP));
     }
 
+    public static String getActualColumnFamilyName(String name) {
+        if(name.startsWith(LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+            return name.substring(LOCAL_INDEX_COLUMN_FAMILY_PREFIX.length());
+        }
+        return name;
+    }
+
     public static String getCaseSensitiveDataColumnFullName(String name) {
         int index = name.indexOf(INDEX_COLUMN_NAME_SEP) ;
-        return SchemaUtil.getCaseSensitiveColumnDisplayName(name.substring(0, index), name.substring(index+1));
+        return SchemaUtil.getCaseSensitiveColumnDisplayName(getDataColumnFamilyName(name), name.substring(index+1));
     }
 
     public static String getIndexColumnName(String dataColumnFamilyName, String dataColumnName) {
-        return (dataColumnFamilyName == null ? "" : dataColumnFamilyName) + INDEX_COLUMN_NAME_SEP + dataColumnName;
+        return (dataColumnFamilyName == null ? "" : dataColumnFamilyName) + INDEX_COLUMN_NAME_SEP
+                + dataColumnName;
     }
     
     public static byte[] getIndexColumnName(byte[] dataColumnFamilyName, byte[] dataColumnName) {
         return ByteUtil.concat(dataColumnFamilyName == null ?  ByteUtil.EMPTY_BYTE_ARRAY : dataColumnFamilyName, INDEX_COLUMN_NAME_SEP_BYTES, dataColumnName);
     }
-    
+
     public static String getIndexColumnName(PColumn dataColumn) {
         String dataColumnFamilyName = SchemaUtil.isPKColumn(dataColumn) ? null : dataColumn.getFamilyName().getString();
         return getIndexColumnName(dataColumnFamilyName, dataColumn.getName().getString());
     }
 
+    public static String getLocalIndexColumnFamily(String dataColumnFamilyName) {
+        return dataColumnFamilyName == null ? null
+                : QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX + dataColumnFamilyName;
+    }
+    
     public static PColumn getDataColumn(PTable dataTable, String indexColumnName) {
         int pos = indexColumnName.indexOf(INDEX_COLUMN_NAME_SEP);
         if (pos < 0) {
@@ -173,7 +186,7 @@ public class IndexUtil {
         }
         PColumnFamily family;
         try {
-            family = dataTable.getColumnFamily(indexColumnName.substring(0, pos));
+            family = dataTable.getColumnFamily(getDataColumnFamilyName(indexColumnName));                
         } catch (ColumnFamilyNotFoundException e) {
             throw new IllegalArgumentException("Could not find column family \"" +  indexColumnName.substring(0, pos) + "\" in index column name of \"" + indexColumnName + "\"", e);
         }
@@ -222,7 +235,14 @@ public class IndexUtil {
             for (final Mutation dataMutation : dataMutations) {
                 long ts = MetaDataUtil.getClientTimeStamp(dataMutation);
                 ptr.set(dataMutation.getRow());
-                Delete delete = maintainer.buildDeleteMutation(kvBuilder, ptr, ts);
+                byte[] regionStartKey = null;
+                byte[] regionEndkey = null;
+                if(maintainer.isLocalIndex()) {
+                    HRegionLocation tableRegionLocation = connection.getQueryServices().getTableRegionLocation(table.getPhysicalName().getBytes(), dataMutation.getRow());
+                    regionStartKey = tableRegionLocation.getRegionInfo().getStartKey();
+                    regionEndkey = tableRegionLocation.getRegionInfo().getEndKey();
+                }
+                Delete delete = maintainer.buildDeleteMutation(kvBuilder, null, ptr, Collections.<KeyValue>emptyList(), ts, regionStartKey, regionEndkey);
                 delete.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, dataMutation.getAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY));
                 indexMutations.add(delete);
             }
@@ -336,55 +356,6 @@ public class IndexUtil {
         });
     }
 
-    public static Region getIndexRegion(RegionCoprocessorEnvironment environment)
-            throws IOException {
-        Region dataRegion = environment.getRegion();
-        return getIndexRegion(dataRegion, environment.getRegionServerServices());
-    }
-
-    public static Region
-            getIndexRegion(Region dataRegion, RegionServerCoprocessorEnvironment env)
-                    throws IOException {
-        return getIndexRegion(dataRegion, env.getRegionServerServices());
-    }
-
-    public static Region getDataRegion(RegionCoprocessorEnvironment env) throws IOException {
-        Region indexRegion = env.getRegion();
-        return getDataRegion(indexRegion, env.getRegionServerServices());
-    }
-
-    public static Region
-            getDataRegion(Region indexRegion, RegionServerCoprocessorEnvironment env)
-                    throws IOException {
-        return getDataRegion(indexRegion, env.getRegionServerServices());
-    }
-
-    public static Region getIndexRegion(Region dataRegion, RegionServerServices rss) throws IOException {
-        TableName indexTableName =
-                TableName.valueOf(MetaDataUtil.getLocalIndexPhysicalName(dataRegion.getTableDesc()
-                        .getName()));
-        List<Region> onlineRegions = rss.getOnlineRegions(indexTableName);
-        for(Region indexRegion : onlineRegions) {
-            if (Bytes.compareTo(dataRegion.getRegionInfo().getStartKey(),
-                    indexRegion.getRegionInfo().getStartKey()) == 0) {
-                return indexRegion;
-            }
-        }
-        return null;
-    }
-
-    public static Region getDataRegion(Region indexRegion, RegionServerServices rss) throws IOException {
-        TableName dataTableName = TableName.valueOf(MetaDataUtil.getUserTableName(indexRegion.getTableDesc().getNameAsString()));
-        List<Region> onlineRegions = rss.getOnlineRegions(dataTableName);
-        for(Region region : onlineRegions) {
-            if (Bytes.compareTo(indexRegion.getRegionInfo().getStartKey(),
-                    region.getRegionInfo().getStartKey()) == 0) {
-                return region;
-            }
-        }
-        return null;
-    }
-
     public static ColumnReference[] deserializeDataTableColumnsToJoin(Scan scan) {
         byte[] columnsBytes = scan.getAttribute(BaseScannerRegionObserver.DATA_TABLE_COLUMNS_TO_JOIN);
         if (columnsBytes == null) return null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
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 116b62b..b65677c 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
@@ -29,6 +29,7 @@ import java.io.IOException;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
@@ -430,15 +431,35 @@ public class MetaDataUtil {
     }
 
     public static boolean hasLocalIndexTable(PhoenixConnection connection, byte[] physicalTableName) throws SQLException {
-        byte[] physicalIndexName = MetaDataUtil.getLocalIndexPhysicalName(physicalTableName);
         try {
-            HTableDescriptor desc = connection.getQueryServices().getTableDescriptor(physicalIndexName);
-            return desc != null && Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(desc.getValue(IS_LOCAL_INDEX_TABLE_PROP_BYTES)));
+            HTableDescriptor desc = connection.getQueryServices().getTableDescriptor(physicalTableName);
+            if(desc == null ) return false;
+            return hasLocalIndexColumnFamily(desc);
         } catch (TableNotFoundException e) {
             return false;
         }
     }
 
+    public static boolean hasLocalIndexColumnFamily(HTableDescriptor desc) {
+        for (HColumnDescriptor cf : desc.getColumnFamilies()) {
+            if (cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    public static List<byte[]> getNonLocalIndexColumnFamilies(HTableDescriptor desc) {
+    	List<byte[]> families = new ArrayList<byte[]>(desc.getColumnFamilies().length);
+        for (HColumnDescriptor cf : desc.getColumnFamilies()) {
+            if (!cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+            	families.add(cf.getName());
+            }
+        }
+    	return families;
+    }
+
+
     public static void deleteViewIndexSequences(PhoenixConnection connection, PName name) throws SQLException {
         String schemaName = getViewIndexSchemaName(name);
         connection.createStatement().executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE + 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index a8981a4..c0fc765 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -517,6 +517,7 @@ public class PhoenixRuntime {
         private boolean isBypassUpgrade;
         private boolean mapNamespace;
         private String srcTable;
+        private boolean localIndexUpgrade;
 
         /**
          * Factory method to build up an {@code ExecutionCommand} based on supplied parameters.
@@ -558,6 +559,9 @@ public class PhoenixRuntime {
             Option mapNamespaceOption = new Option("m", "map-namespace", true,
                     "Used to map table to a namespace matching with schema, require "+ QueryServices.IS_NAMESPACE_MAPPING_ENABLED +
                     " to be enabled");
+            Option localIndexUpgradeOption = new Option("l", "local-index-upgrade", false,
+                "Used to upgrade local index data by moving index data from separate table to "
+                + "separate column families in the same table.");
             Options options = new Options();
             options.addOption(tableOption);
             options.addOption(headerOption);
@@ -569,6 +573,7 @@ public class PhoenixRuntime {
             options.addOption(upgradeOption);
             options.addOption(bypassUpgradeOption);
             options.addOption(mapNamespaceOption);
+            options.addOption(localIndexUpgradeOption);
 
             CommandLineParser parser = new PosixParser();
             CommandLine cmdLine = null;
@@ -622,7 +627,7 @@ public class PhoenixRuntime {
                 }
                 execCmd.isBypassUpgrade = true;
             }
-
+            execCmd.localIndexUpgrade = cmdLine.hasOption(localIndexUpgradeOption.getOpt());
 
             List<String> argList = Lists.newArrayList(cmdLine.getArgList());
             if (argList.isEmpty()) {
@@ -737,6 +742,10 @@ public class PhoenixRuntime {
         public String getSrcTable() {
             return srcTable;
         }
+
+        public boolean isLocalIndexUpgrade() {
+            return localIndexUpgrade;
+        }
     }
     
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
----------------------------------------------------------------------
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 8d00b2b..474cf34 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
@@ -70,6 +70,8 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.regionserver.LocalIndexSplitter;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataEndpointImpl;
@@ -292,6 +294,119 @@ public class UpgradeUtil {
             }
         }
     }
+
+    public static void upgradeLocalIndexes(PhoenixConnection metaConnection, boolean createAsyncIndex) throws SQLException,
+            IOException, org.apache.hadoop.hbase.TableNotFoundException {
+        HBaseAdmin admin = null;
+        try {
+            admin = metaConnection.getQueryServices().getAdmin();
+            ResultSet rs = metaConnection.createStatement().executeQuery("SELECT TABLE_SCHEM, TABLE_NAME, DATA_TABLE_NAME FROM SYSTEM.CATALOG  "
+                    + "      WHERE COLUMN_NAME IS NULL"
+                    + "           AND COLUMN_FAMILY IS NULL"
+                    + "           AND INDEX_TYPE=2");
+            boolean droppedLocalIndexes = false;
+            while (rs.next()) {
+                if(!droppedLocalIndexes) {
+                    HTableDescriptor[] localIndexTables = admin.listTables(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX+".*");
+                    String localIndexSplitter = LocalIndexSplitter.class.getName();
+                    for (HTableDescriptor table : localIndexTables) {
+                        HTableDescriptor dataTableDesc = admin.getTableDescriptor(TableName.valueOf(MetaDataUtil.getUserTableName(table.getNameAsString())));
+                        HColumnDescriptor[] columnFamilies = dataTableDesc.getColumnFamilies();
+                        boolean modifyTable = false;
+                        for(HColumnDescriptor cf : columnFamilies) {
+                            String localIndexCf = QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX+cf.getNameAsString();
+                            if(dataTableDesc.getFamily(Bytes.toBytes(localIndexCf))==null){
+                                HColumnDescriptor colDef =
+                                        new HColumnDescriptor(localIndexCf);
+                                for(Entry<ImmutableBytesWritable, ImmutableBytesWritable>keyValue: cf.getValues().entrySet()){
+                                    colDef.setValue(keyValue.getKey().copyBytes(), keyValue.getValue().copyBytes());
+                                }
+                                dataTableDesc.addFamily(colDef);
+                                modifyTable = true;
+                            }
+                        }
+                        List<String> coprocessors = dataTableDesc.getCoprocessors();
+                        for(String coprocessor:  coprocessors) {
+                            if(coprocessor.equals(localIndexSplitter)) {
+                                dataTableDesc.removeCoprocessor(localIndexSplitter);
+                                modifyTable = true;
+                            }
+                        }
+                        if(modifyTable) {
+                            admin.modifyTable(dataTableDesc.getName(), dataTableDesc);
+                        }
+                    }
+                    admin.disableTables(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX+".*");
+                    admin.deleteTables(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX+".*");
+                    droppedLocalIndexes = true;
+                }
+                String getColumns =
+                        "SELECT COLUMN_NAME, COLUMN_FAMILY FROM SYSTEM.CATALOG  WHERE TABLE_SCHEM "
+                                + (rs.getString(1) == null ? "IS NULL " : "='" + rs.getString(1)
+                                + "'") + " and TABLE_NAME='" + rs.getString(2)
+                                + "' AND COLUMN_NAME IS NOT NULL";
+                ResultSet getColumnsRs = metaConnection.createStatement().executeQuery(getColumns);
+                List<String> indexedColumns = new ArrayList<String>(1);
+                List<String> coveredColumns = new ArrayList<String>(1);
+                
+                while (getColumnsRs.next()) {
+                    String column = getColumnsRs.getString(1);
+                    String columnName = IndexUtil.getDataColumnName(column);
+                    if (columnName.equals(MetaDataUtil.getViewIndexIdColumnName())) {
+                        continue;
+                    }
+                    String columnFamily = IndexUtil.getDataColumnFamilyName(column);
+                    if (getColumnsRs.getString(2) == null) {
+                        if (columnFamily != null && !columnFamily.isEmpty()) {
+                            if (columnFamily.equals(QueryConstants.DEFAULT_COLUMN_FAMILY)) {
+                                indexedColumns.add(columnName);
+                            } else {
+                                indexedColumns.add(SchemaUtil.getColumnName(columnFamily,
+                                    columnName));
+                            }
+                        }
+                    } else {
+                        coveredColumns.add(SchemaUtil.getColumnName(columnFamily, columnName));
+                    }
+                }
+                StringBuilder createIndex = new StringBuilder("CREATE LOCAL INDEX ");
+                createIndex.append(rs.getString(2));
+                createIndex.append(" ON ");
+                createIndex.append(SchemaUtil.getTableName(rs.getString(1), rs.getString(3)));
+                createIndex.append("(");
+                for (int i = 0; i < indexedColumns.size(); i++) {
+                    createIndex.append(indexedColumns.get(i));
+                    if (i < indexedColumns.size() - 1) {
+                        createIndex.append(",");
+                    }
+                }
+                createIndex.append(")");
+               
+                if (!coveredColumns.isEmpty()) {
+                    createIndex.append(" INCLUDE(");
+                    for (int i = 0; i < coveredColumns.size(); i++) {
+                        createIndex.append(coveredColumns.get(i));
+                        if (i < coveredColumns.size() - 1) {
+                            createIndex.append(",");
+                        }
+                    }
+                    createIndex.append(") ASYNC");
+                }
+                logger.info("Index creation query is : " + createIndex.toString());
+                logger.info("Dropping the index " + rs.getString(2)
+                    + " to clean up the index details from SYSTEM.CATALOG.");
+                metaConnection.createStatement().execute(
+                    "DROP INDEX IF EXISTS " + rs.getString(2) + " ON "
+                            + SchemaUtil.getTableName(rs.getString(1), rs.getString(3)));
+                logger.info("Recreating the index " + rs.getString(2));
+                metaConnection.createStatement().execute(createIndex.toString());
+                logger.info("Created the index " + rs.getString(2));
+            }
+            metaConnection.createStatement().execute("DELETE FROM SYSTEM.CATALOG WHERE SUBSTR(TABLE_NAME,0,11)='"+MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX+"'");
+        } finally {
+            if (admin != null) admin.close();
+        }
+    }
     
     @SuppressWarnings("deprecation")
     public static boolean upgradeSequenceTable(PhoenixConnection conn, int nSaltBuckets, PTable oldTable) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleIndexWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleIndexWriter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleIndexWriter.java
index 88bf7fc..1f1e37e 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleIndexWriter.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleIndexWriter.java
@@ -108,7 +108,7 @@ public class TestParalleIndexWriter {
     // setup the writer and failure policy
     ParallelWriterIndexCommitter writer = new ParallelWriterIndexCommitter(VersionInfo.getVersion());
     writer.setup(factory, exec, abort, stop, 1);
-    writer.write(indexUpdates);
+    writer.write(indexUpdates, true);
     assertTrue("Writer returned before the table batch completed! Likely a race condition tripped",
       completed[0]);
     writer.stop(this.test.getTableNameString() + " finished");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleWriterIndexCommitter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleWriterIndexCommitter.java
index ee5e1d5..8eece3b 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleWriterIndexCommitter.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleWriterIndexCommitter.java
@@ -108,7 +108,7 @@ public class TestParalleWriterIndexCommitter {
     // setup the writer and failure policy
     ParallelWriterIndexCommitter writer = new ParallelWriterIndexCommitter(VersionInfo.getVersion());
     writer.setup(factory, exec, abort, stop, 1);
-    writer.write(indexUpdates);
+    writer.write(indexUpdates, true);
     assertTrue("Writer returned before the table batch completed! Likely a race condition tripped",
       completed[0]);
     writer.stop(this.test.getTableNameString() + " finished");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index 294c82f..fdb4002 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -128,22 +128,16 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.ipc.controller.ServerRpcControllerFactory;
-import org.apache.hadoop.hbase.master.LoadBalancer;
-import org.apache.hadoop.hbase.regionserver.LocalIndexMerger;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
-import org.apache.phoenix.hbase.index.balancer.IndexLoadBalancer;
-import org.apache.phoenix.hbase.index.master.IndexMasterObserver;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixDriver;
@@ -748,11 +742,6 @@ public abstract class BaseTest {
         conf.setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 5);
         conf.setInt("hbase.regionserver.metahandler.count", 2);
         conf.setInt(HConstants.MASTER_HANDLER_COUNT, 2);
-        conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, IndexMasterObserver.class.getName());
-        conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, IndexLoadBalancer.class,
-            LoadBalancer.class);
-        conf.setClass("hbase.coprocessor.regionserver.classes", LocalIndexMerger.class,
-            RegionServerObserver.class) ;
         conf.setInt("dfs.namenode.handler.count", 2);
         conf.setInt("dfs.namenode.service.handler.count", 2);
         conf.setInt("dfs.datanode.handler.count", 2);


[5/5] phoenix git commit: PHOENIX-1734 Local index improvements(Rajeshbabu)

Posted by ra...@apache.org.
PHOENIX-1734 Local index improvements(Rajeshbabu)


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

Branch: refs/heads/master
Commit: 10909ae502095bac775d98e6d92288c5cad9b9a6
Parents: 18da4a0
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Tue May 24 21:04:04 2016 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Tue May 24 21:04:04 2016 +0530

----------------------------------------------------------------------
 ...ReplayWithIndexWritesAndCompressedWALIT.java |   2 +-
 .../phoenix/end2end/AggregateQueryIT.java       |  18 +-
 .../end2end/BaseTenantSpecificViewIndexIT.java  |   6 +-
 .../org/apache/phoenix/end2end/BaseViewIT.java  |   6 +-
 .../org/apache/phoenix/end2end/DeleteIT.java    |  11 +-
 .../org/apache/phoenix/end2end/HashJoinIT.java  |  55 +-
 .../phoenix/end2end/HashJoinLocalIndexIT.java   |  13 +-
 .../org/apache/phoenix/end2end/IndexToolIT.java |   5 +-
 .../apache/phoenix/end2end/SortMergeJoinIT.java |  11 +-
 .../org/apache/phoenix/end2end/SubqueryIT.java  |  19 +-
 .../end2end/SubqueryUsingSortMergeJoinIT.java   |  12 +-
 .../end2end/TenantSpecificViewIndexIT.java      |   2 +-
 .../org/apache/phoenix/end2end/UpgradeIT.java   |   3 +-
 .../phoenix/end2end/UserDefinedFunctionsIT.java |   2 +-
 .../java/org/apache/phoenix/end2end/ViewIT.java |   2 +-
 .../phoenix/end2end/index/DropMetadataIT.java   |   3 +-
 .../end2end/index/IndexExpressionIT.java        |  32 +-
 .../apache/phoenix/end2end/index/IndexIT.java   |  20 +-
 .../phoenix/end2end/index/LocalIndexIT.java     | 190 +---
 .../end2end/index/MutableIndexFailureIT.java    |  16 +-
 .../phoenix/end2end/index/MutableIndexIT.java   | 104 +-
 .../end2end/index/ReadOnlyIndexFailureIT.java   |  10 +
 .../phoenix/end2end/index/ViewIndexIT.java      |   4 +-
 .../end2end/index/txn/MutableRollbackIT.java    |  24 +-
 .../phoenix/end2end/index/txn/RollbackIT.java   |   2 +-
 .../end2end/index/txn/TxWriteFailureIT.java     |   2 +-
 .../index/balancer/IndexLoadBalancerIT.java     | 485 ---------
 .../IndexHalfStoreFileReaderGenerator.java      |  17 +-
 .../regionserver/IndexSplitTransaction.java     | 986 -------------------
 .../hbase/regionserver/LocalIndexMerger.java    | 123 ---
 .../hbase/regionserver/LocalIndexSplitter.java  | 137 ---
 .../phoenix/compile/CreateTableCompiler.java    |  11 +
 .../phoenix/compile/ProjectionCompiler.java     |  10 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |  11 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   1 +
 .../GroupedAggregateRegionObserver.java         |   5 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  14 +-
 .../phoenix/coprocessor/ScanRegionObserver.java |   2 +-
 .../UngroupedAggregateRegionObserver.java       |  45 +-
 .../phoenix/exception/SQLExceptionCode.java     |   2 +
 .../hbase/index/IndexRegionSplitPolicy.java     |  15 +-
 .../org/apache/phoenix/hbase/index/Indexer.java |  49 +-
 .../hbase/index/balancer/IndexLoadBalancer.java | 671 +------------
 .../hbase/index/master/IndexMasterObserver.java |  86 --
 .../hbase/index/write/IndexCommitter.java       |   2 +-
 .../phoenix/hbase/index/write/IndexWriter.java  |  27 +-
 .../write/ParallelWriterIndexCommitter.java     |  38 +-
 .../TrackingParallelWriterIndexCommitter.java   |  36 +-
 .../apache/phoenix/index/IndexMaintainer.java   |  38 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |   6 +-
 .../index/PhoenixIndexFailurePolicy.java        |   5 +-
 .../index/PhoenixTransactionalIndexer.java      |   5 +-
 .../apache/phoenix/iterate/ExplainTable.java    |  21 +-
 .../phoenix/mapreduce/AbstractBulkLoadTool.java |   9 +-
 .../phoenix/mapreduce/index/IndexTool.java      |   3 +-
 .../query/ConnectionQueryServicesImpl.java      | 144 ++-
 .../apache/phoenix/query/QueryConstants.java    |  11 +
 .../apache/phoenix/schema/MetaDataClient.java   |  62 +-
 .../org/apache/phoenix/schema/TableRef.java     |   3 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |  83 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |  27 +-
 .../org/apache/phoenix/util/PhoenixRuntime.java |  11 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    | 115 +++
 .../index/write/TestParalleIndexWriter.java     |   2 +-
 .../write/TestParalleWriterIndexCommitter.java  |   2 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  11 -
 66 files changed, 742 insertions(+), 3163 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java b/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
index 14b6fdf..0b48a1a 100644
--- a/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
+++ b/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
@@ -83,7 +83,7 @@ import org.mockito.Mockito;
 @Category(NeedsOwnMiniClusterTest.class)
 public class WALReplayWithIndexWritesAndCompressedWALIT {
 
-  public static final Log LOG = LogFactory.getLog(TestWALReplay.class);
+  public static final Log LOG = LogFactory.getLog(WALReplayWithIndexWritesAndCompressedWALIT.class);
   @Rule
   public TableName table = new TableName();
   private String INDEX_TABLE_NAME = table.getTableNameString() + "_INDEX";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateQueryIT.java
index 4eb9c13..b92e8b8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateQueryIT.java
@@ -145,16 +145,14 @@ public class AggregateQueryIT extends BaseQueryIT {
             HTable htable = (HTable) conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(tableName);
             htable.clearRegionCache();
             int nRegions = htable.getRegionLocations().size();
-            if(!admin.tableExists(TableName.valueOf(MetaDataUtil.getLocalIndexTableName(ATABLE_NAME)))) {
-                admin.split(tableName, ByteUtil.concat(Bytes.toBytes(tenantId), Bytes.toBytes("00A" + Character.valueOf((char) ('3' + nextRunCount())) + ts))); // vary split point with test run
-                int retryCount = 0;
-                do {
-                    Thread.sleep(2000);
-                    retryCount++;
-                    //htable.clearRegionCache();
-                } while (retryCount < 10 && htable.getRegionLocations().size() == nRegions);
-                assertNotEquals(nRegions, htable.getRegionLocations().size());
-            } 
+            admin.split(tableName, ByteUtil.concat(Bytes.toBytes(tenantId), Bytes.toBytes("00A" + Character.valueOf((char) ('3' + nextRunCount())) + ts))); // vary split point with test run
+            int retryCount = 0;
+            do {
+                Thread.sleep(2000);
+                retryCount++;
+                //htable.clearRegionCache();
+            } while (retryCount < 10 && htable.getRegionLocations().size() == nRegions);
+            assertNotEquals(nRegions, htable.getRegionLocations().size());
             
             statement.setString(1, tenantId);
             rs = statement.executeQuery();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
----------------------------------------------------------------------
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 c10afa6..9f25531 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
@@ -132,10 +132,10 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
         ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT k1, k2, v2 FROM v WHERE v2='" + valuePrefix + "v2-1'");
         if(localIndex){
             assertEquals(saltBuckets == null ? 
-                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_T ['" + tenantId + "',-32768,'" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER T ['" + tenantId + "',1,'" + valuePrefix + "v2-1']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY\n"
                             + "CLIENT MERGE SORT" :
-                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER _LOCAL_IDX_T ['" + tenantId + "',-32768,'" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER T ['" + tenantId + "',1,'" + valuePrefix + "v2-1']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY\n"
                             + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
         } else {
@@ -155,7 +155,7 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
         conn.commit();
         ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT k1, k2, v2 FROM v WHERE v2='" + valuePrefix + "v2-1'");
         assertEquals(
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_T [" + tenantId + ",-32768,'" + valuePrefix + "v2-1']\n"
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER T [" + tenantId + ",1,'" + valuePrefix + "v2-1']\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY\n"
                         + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
----------------------------------------------------------------------
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 65f1f93..6d8d889 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
@@ -203,7 +203,7 @@ public abstract class BaseViewIT extends BaseOwnClusterHBaseManagedTimeIT {
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         String queryPlan = QueryUtil.getExplainPlan(rs);
         if (localIndex) {
-            assertEquals("CLIENT PARALLEL "+ (saltBuckets == null ? 1 : saltBuckets)  +"-WAY RANGE SCAN OVER _LOCAL_IDX_" + tableName +" [-32768,51]\n"
+            assertEquals("CLIENT PARALLEL "+ (saltBuckets == null ? 1 : saltBuckets)  +"-WAY RANGE SCAN OVER " + tableName +" [1,51]\n"
                     + "    SERVER FILTER BY FIRST KEY ONLY\n"
                     + "CLIENT MERGE SORT",
                 queryPlan);
@@ -242,8 +242,8 @@ public abstract class BaseViewIT extends BaseOwnClusterHBaseManagedTimeIT {
         String htableName;
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         if (localIndex) {
-            htableName = "_LOCAL_IDX_" + tableName;
-            assertEquals("CLIENT PARALLEL "+ (saltBuckets == null ? 1 : saltBuckets)  +"-WAY RANGE SCAN OVER " + htableName +" [" + (Short.MIN_VALUE+1) + ",'foo']\n"
+            htableName = tableName;
+            assertEquals("CLIENT PARALLEL "+ (saltBuckets == null ? 1 : saltBuckets)  +"-WAY RANGE SCAN OVER " + htableName +" [" + (2) + ",'foo']\n"
                     + "    SERVER FILTER BY FIRST KEY ONLY\n"
                     + "CLIENT MERGE SORT",QueryUtil.getExplainPlan(rs));
         } else {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
index 10152e3..da75476 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
@@ -33,7 +33,6 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.junit.Test;
 
@@ -161,7 +160,7 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
         if (createIndex) {
             if (local) {
                 conn.createStatement().execute("CREATE LOCAL INDEX IF NOT EXISTS local_idx ON IntIntKeyTest(j)");
-                indexName = MetaDataUtil.getLocalIndexTableName("INTINTKEYTEST");
+                indexName = "INTINTKEYTEST";
             } else {
                 conn.createStatement().execute("CREATE INDEX IF NOT EXISTS idx ON IntIntKeyTest(j)");
             }
@@ -187,7 +186,9 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
         PreparedStatement stmt;
         conn.setAutoCommit(autoCommit);
         deleteStmt = "DELETE FROM IntIntKeyTest WHERE i >= ? and i < ?";
-        assertIndexUsed(conn, deleteStmt, Arrays.<Object>asList(5,10), indexName, false);
+        if(!local) {
+            assertIndexUsed(conn, deleteStmt, Arrays.<Object>asList(5,10), indexName, false);
+        }
         stmt = conn.prepareStatement(deleteStmt);
         stmt.setInt(1, 5);
         stmt.setInt(2, 10);
@@ -205,7 +206,9 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
         
         deleteStmt = "DELETE FROM IntIntKeyTest WHERE j IS NULL";
         stmt = conn.prepareStatement(deleteStmt);
-        assertIndexUsed(conn, deleteStmt, indexName, createIndex);
+        if(!local) {
+            assertIndexUsed(conn, deleteStmt, indexName, createIndex);
+        }
         int deleteCount = stmt.executeUpdate();
         assertEquals(3, deleteCount);
         if (!autoCommit) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
index 3e2356f..9cd8a5f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
@@ -53,7 +53,6 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -851,7 +850,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "    SERVER AGGREGATE INTO DISTINCT ROWS BY [\"I.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_ITEM_TABLE_DISPLAY_NAME +" [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +JOIN_ITEM_TABLE_DISPLAY_NAME +" [1]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        CLIENT MERGE SORT",
                 /* 
@@ -865,7 +864,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "CLIENT MERGE SORT\n" +
                 "CLIENT SORTED BY [SUM(O.QUANTITY) DESC]\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_ITEM_TABLE_DISPLAY_NAME +" [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME +" [1]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        CLIENT MERGE SORT",          
                 /* 
@@ -886,7 +885,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     RIGHT JOIN joinItemTable i ON o.item_id = i.item_id 
                  *     GROUP BY i.name ORDER BY i.name
                  */
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_ITEM_TABLE_DISPLAY_NAME+" [-32768]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER "+ JOIN_ITEM_TABLE_DISPLAY_NAME+" [1]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [\"I.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" + 
@@ -921,11 +920,11 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *         AND (supp.name BETWEEN 'S1' AND 'S5') 
                  *     WHERE item.name BETWEEN 'T1' AND 'T5'
                  */
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768,'T1'] - [-32768,'T5']\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1,'T1'] - [1,'T5']\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_SUPPLIER_TABLE_DISPLAY_NAME +" [-32768,'S1'] - [-32768,'S5']\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME +" [1,'S1'] - [1,'S5']\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "        CLIENT MERGE SORT",
                 /*
@@ -936,10 +935,10 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     WHERE (item.name = 'T1' OR item.name = 'T5') 
                  *         AND (supp.name = 'S1' OR supp.name = 'S5')
                  */
-                "CLIENT PARALLEL 1-WAY SKIP SCAN ON 2 KEYS OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768,'T1'] - [-32768,'T5']\n" +
+                "CLIENT PARALLEL 1-WAY SKIP SCAN ON 2 KEYS OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1,'T1'] - [1,'T5']\n" +
                 "CLIENT MERGE SORT\n" + 
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY SKIP SCAN ON 2 KEYS OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_SUPPLIER_TABLE_DISPLAY_NAME +" [-32768,'S1'] - [-32768,'S5']\n" + 
+                "        CLIENT PARALLEL 1-WAY SKIP SCAN ON 2 KEYS OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME +" [1,'S1'] - [1,'S5']\n" + 
                 "            SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "        CLIENT MERGE SORT",
                 /*
@@ -948,13 +947,13 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     JOIN joinOrderTable o ON o.item_id = i.item_id AND quantity < 5000 
                  *     JOIN joinSupplierTable s ON i.supplier_id = s.supplier_id
                  */
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "CLIENT MERGE SORT\n" + 
                 "    PARALLEL INNER-JOIN TABLE 0 (SKIP MERGE)\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "            SERVER FILTER BY QUANTITY < 5000\n" +
                 "    PARALLEL INNER-JOIN TABLE 1\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [1]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "        CLIENT MERGE SORT\n" +
                 "    DYNAMIC SERVER FILTER BY \"I.:item_id\" IN (\"O.item_id\")",
@@ -966,7 +965,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  */
                 "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + "\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER "+ MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+ JOIN_ITEM_TABLE_DISPLAY_NAME +" [-32768]\n"  +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER "+ JOIN_ITEM_TABLE_DISPLAY_NAME +" [1]\n"  +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        CLIENT MERGE SORT\n" +
                 "    DYNAMIC SERVER FILTER BY \"I1.item_id\" IN (\"I2.:item_id\")",
@@ -976,12 +975,12 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     JOIN joinItemTable i2 ON i1.item_id = i2.supplier_id 
                  *     ORDER BY i1.name, i2.name
                  */
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+ JOIN_ITEM_TABLE_DISPLAY_NAME +" [-32768]\n"  +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME +" [1]\n"  +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "    SERVER SORTED BY [\"I1.0:NAME\", \"I2.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+ JOIN_ITEM_TABLE_DISPLAY_NAME +" [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME +" [1]\n" +
                 "        CLIENT MERGE SORT\n" +
                 "    DYNAMIC SERVER FILTER BY \"I1.:item_id\" IN (\"I2.0:supplier_id\")",
                 /*
@@ -994,11 +993,11 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  */
                 "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_CUSTOMER_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_CUSTOMER_TABLE_DISPLAY_NAME + " [1]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "        CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 1\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        CLIENT MERGE SORT",
                 /*
@@ -1009,14 +1008,14 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     JOIN joinItemTable i ON o.item_id = i.item_id 
                  *     ORDER BY order_id
                  */
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "    SERVER SORTED BY [\"O.order_id\"]\n"+
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "            PARALLEL INNER-JOIN TABLE 0\n" +
-                "                CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + "" + JOIN_CUSTOMER_TABLE_DISPLAY_NAME+" [-32768]\n"+
+                "                CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_CUSTOMER_TABLE_DISPLAY_NAME+" [1]\n"+
                 "                    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "                CLIENT MERGE SORT\n" +
                 "    DYNAMIC SERVER FILTER BY \"I.:item_id\" IN (\"O.item_id\")",
@@ -1040,7 +1039,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "            SERVER FILTER BY \"order_id\" != '000000000000003'\n" +
                 "            PARALLEL INNER-JOIN TABLE 0\n" +
-                "                CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+ JOIN_ITEM_TABLE_DISPLAY_NAME +" [-32768]\n" +
+                "                CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME +" [1]\n" +
                 "                    SERVER FILTER BY \"NAME\" != 'T3'\n" +
                 "                CLIENT MERGE SORT\n" +
                 "                    PARALLEL LEFT-JOIN TABLE 0\n" +
@@ -1057,7 +1056,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "    SERVER AGGREGATE INTO DISTINCT ROWS BY [I.NAME]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+JOIN_ITEM_TABLE_DISPLAY_NAME+" [-32768]\n"+
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +JOIN_ITEM_TABLE_DISPLAY_NAME+" [1]\n"+
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        CLIENT MERGE SORT",
                 /* 
@@ -1073,7 +1072,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "CLIENT MERGE SORT\n" +
                 "CLIENT SORTED BY [SUM(O.QUANTITY) DESC]\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0 (SKIP MERGE)\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        CLIENT MERGE SORT",
                 /* 
@@ -1084,7 +1083,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     ON o.iid = i.iid 
                  *     ORDER BY o.q DESC NULLS LAST, i.iid
                  */     
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "    SERVER SORTED BY [O.Q DESC NULLS LAST, I.IID]\n"+
                 "CLIENT MERGE SORT\n" +
@@ -1100,7 +1099,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     ON o.iid = i.iid 
                  *     ORDER BY o.q DESC, i.iid
                  */     
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "    SERVER SORTED BY [O.Q DESC, I.IID]\n"+
                 "CLIENT MERGE SORT\n" +
@@ -1134,7 +1133,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
                 "            SERVER FILTER BY \"order_id\" != '000000000000003'\n" +
                 "            PARALLEL INNER-JOIN TABLE 0\n" +
-                "                CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +  MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "                CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +  JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "                    SERVER FILTER BY \"NAME\" != 'T3'\n" +
                 "                CLIENT MERGE SORT\n" +      
                 "                    PARALLEL LEFT-JOIN TABLE 0\n" +
@@ -1150,7 +1149,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "    SERVER 4 ROW LIMIT\n" +
                 "CLIENT 4 ROW LIMIT\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER "+ MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER "+ JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "        CLIENT MERGE SORT\n" +      
                 "    PARALLEL LEFT-JOIN TABLE 1(DELAYED EVALUATION)\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER "+ JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
@@ -1165,7 +1164,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + "\n" +
                 "CLIENT 4 ROW LIMIT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER "+ MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER "+ JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "        CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 1(DELAYED EVALUATION)\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER "+ JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
@@ -1180,7 +1179,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                  *     JOIN (SELECT order_id, item_id, quantity FROM joinOrderTable) o
                  *     ON o.item_id = i.item_id;
                  */
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "CLIENT MERGE SORT\n" +
                 "CLIENT 4 ROW LIMIT\n" +
@@ -1200,7 +1199,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "    SERVER 3 ROW LIMIT\n" +
                 "CLIENT 1 ROW LIMIT\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER "+ MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER "+ JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "        CLIENT MERGE SORT\n" +      
                 "    PARALLEL LEFT-JOIN TABLE 1(DELAYED EVALUATION)\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER "+ JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
@@ -1216,7 +1215,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
                 "    SERVER OFFSET 2\n" +
                 "CLIENT 1 ROW LIMIT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER "+ MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX +""+JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER "+ JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "        CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 1(DELAYED EVALUATION)\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER "+ JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinLocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinLocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinLocalIndexIT.java
index 645d21b..aa99886 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinLocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinLocalIndexIT.java
@@ -38,7 +38,6 @@ import java.util.Properties;
 
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -98,30 +97,30 @@ public class HashJoinLocalIndexIT extends BaseHBaseManagedTimeIT {
                 "CREATE LOCAL INDEX \"idx_item\" ON " + JOIN_ITEM_TABLE_FULL_NAME + " (name)",
                 "CREATE LOCAL INDEX \"idx_supplier\" ON " + JOIN_SUPPLIER_TABLE_FULL_NAME + " (name)"
                 }, {
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [-32768,'S1']\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [1,'S1']\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768,*] - [-32768,'T6']\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1,*] - [1,'T6']\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        CLIENT MERGE SORT\n" +
                 "    DYNAMIC SERVER FILTER BY \"S.:supplier_id\" IN (\"I.supplier_id\")",
                 
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [-32768,'S1']\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [1,'S1']\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "    SERVER AGGREGATE INTO DISTINCT ROWS BY [\"S.PHONE\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768,*] - [-32768,'T6']\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1,*] - [1,'T6']\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "        CLIENT MERGE SORT\n" +
                 "    DYNAMIC SERVER FILTER BY \"S.:supplier_id\" IN (\"I.supplier_id\")",
                 
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [-32768,*] - [-32768,'S3']\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [1,*] - [1,'S3']\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "    SERVER AGGREGATE INTO SINGLE ROW\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768,*] - [-32768,'T6']\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1,*] - [1,'T6']\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "        CLIENT MERGE SORT",
                 }});

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
----------------------------------------------------------------------
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 ef22094..9fb9e0a 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
@@ -39,7 +39,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.mapreduce.index.IndexTool;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -204,8 +203,8 @@ public class IndexToolIT extends BaseOwnClusterHBaseManagedTimeIT {
         
         String expectedExplainPlan = "";
         if(isLocal) {
-            final String localIndexName = MetaDataUtil.getLocalIndexTableName(SchemaUtil.getTableName(schemaName, dataTable));
-            expectedExplainPlan = String.format("CLIENT 1-CHUNK PARALLEL 1-WAY ROUND ROBIN RANGE SCAN OVER %s [-32768]"
+            final String localIndexName = SchemaUtil.getTableName(schemaName, dataTable);
+            expectedExplainPlan = String.format("CLIENT 1-CHUNK PARALLEL 1-WAY ROUND ROBIN RANGE SCAN OVER %s [1]"
                 + "\n    SERVER FILTER BY FIRST KEY ONLY", localIndexName);
         } else {
             expectedExplainPlan = String.format("CLIENT 1-CHUNK PARALLEL 1-WAY ROUND ROBIN FULL SCAN OVER %s"

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
index 43afd0d..38ee4cd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
@@ -51,7 +51,6 @@ import java.util.Properties;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -187,13 +186,13 @@ public class SortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "CREATE LOCAL INDEX \"idx_supplier\" ON " + JOIN_SUPPLIER_TABLE_FULL_NAME + " (name)"
                 }, {
                 "SORT-MERGE-JOIN (LEFT) TABLES\n" +
-                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [1]\n" +
                 "        SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "        SERVER SORTED BY [\"S.:supplier_id\"]\n" +
                 "    CLIENT MERGE SORT\n" +
                 "AND\n" +
                 "    SORT-MERGE-JOIN (INNER) TABLES\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "            SERVER SORTED BY [\"I.:item_id\"]\n" +
                 "        CLIENT MERGE SORT\n" +
                 "    AND (SKIP MERGE)\n" +
@@ -204,7 +203,7 @@ public class SortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "    CLIENT SORTED BY [\"I.0:supplier_id\"]",
                 
                 "SORT-MERGE-JOIN (INNER) TABLES\n" +
-                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "        SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        SERVER SORTED BY [\"I.:item_id\"]\n" +
                 "    CLIENT MERGE SORT\n" +
@@ -215,12 +214,12 @@ public class SortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "CLIENT 4 ROW LIMIT",
                 
                 "SORT-MERGE-JOIN (INNER) TABLES\n" +
-                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_Join.ItemTable [-32768]\n" +
+                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER Join.ItemTable [1]\n" +
                 "        SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        SERVER SORTED BY [\"I1.:item_id\"]\n" +
                 "    CLIENT MERGE SORT\n" +
                 "AND\n" +
-                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_Join.ItemTable [-32768]\n" +
+                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER Join.ItemTable [1]\n" +
                 "        SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        SERVER SORTED BY [\"I2.:item_id\"]\n" +
                 "    CLIENT MERGE SORT\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
index a8b3572..2ced621 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
@@ -48,7 +48,6 @@ import java.util.regex.Pattern;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -245,10 +244,10 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "CREATE LOCAL INDEX \"idx_item\" ON " + JOIN_ITEM_TABLE_FULL_NAME + " (name) INCLUDE (price, discount1, discount2, \"supplier_id\", description)",
                 "CREATE LOCAL INDEX \"idx_supplier\" ON " + JOIN_SUPPLIER_TABLE_FULL_NAME + " (name)"
                 }, {
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[1\\]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL INNER-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " \\[1\\]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "        CLIENT MERGE SORT\n" +
                 "    PARALLEL SEMI-JOIN TABLE 1 \\(SKIP MERGE\\)\n" +
@@ -257,12 +256,12 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "        CLIENT MERGE SORT\n" +
                 "    DYNAMIC SERVER FILTER BY \"I.:item_id\" IN \\(\\$\\d+.\\$\\d+\\)",
                             
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [1]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "    SERVER SORTED BY [\"I.0:NAME\"]\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "        CLIENT MERGE SORT\n" +
                 "    PARALLEL SEMI-JOIN TABLE 1(DELAYED EVALUATION) (SKIP MERGE)\n" +
                 "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + JOIN_ORDER_TABLE_DISPLAY_NAME + "\n" +
@@ -272,7 +271,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "CLIENT PARALLEL 4-WAY FULL SCAN OVER " + JOIN_COITEM_TABLE_DISPLAY_NAME + "\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL LEFT-JOIN TABLE 0\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[1\\]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\".+.:item_id\", \".+.0:NAME\"\\]\n" +
                 "        CLIENT MERGE SORT\n" + 
@@ -281,7 +280,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "                    SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"item_id\"\\]\n" +
                 "                CLIENT MERGE SORT\n" +
                 "    PARALLEL LEFT-JOIN TABLE 1\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[1\\]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\".+.:item_id\", \".+.0:NAME\"\\]\n" +
                 "        CLIENT MERGE SORT\n" + 
@@ -292,7 +291,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "            DYNAMIC SERVER FILTER BY \"" + JOIN_SCHEMA + ".idx_item.:item_id\" IN \\(\\$\\d+.\\$\\d+\\)\n" +
                 "    AFTER-JOIN SERVER FILTER BY \\(\\$\\d+.\\$\\d+ IS NOT NULL OR \\$\\d+.\\$\\d+ IS NOT NULL\\)",
                 
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL ANTI-JOIN TABLE 0 (SKIP MERGE)\n" +
@@ -300,11 +299,11 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
                 "            SERVER AGGREGATE INTO DISTINCT ROWS BY [\"item_id\"]\n" +
                 "        CLIENT MERGE SORT",
                 
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_CUSTOMER_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_CUSTOMER_TABLE_DISPLAY_NAME + " \\[1\\]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "CLIENT MERGE SORT\n" +
                 "    PARALLEL SEMI-JOIN TABLE 0 \\(SKIP MERGE\\)\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[1\\]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "            SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"O.customer_id\"\\]\n" +
                 "        CLIENT MERGE SORT\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryUsingSortMergeJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryUsingSortMergeJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryUsingSortMergeJoinIT.java
index f7a798e..435c649 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryUsingSortMergeJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryUsingSortMergeJoinIT.java
@@ -232,11 +232,11 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 }, {
                 "SORT-MERGE-JOIN (SEMI) TABLES\n" +
                 "    SORT-MERGE-JOIN (INNER) TABLES\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " [1]\n" +
                 "            SERVER SORTED BY [\"I.0:supplier_id\"]\n" +
                 "        CLIENT MERGE SORT\n" +
                 "    AND\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [-32768]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_SUPPLIER_TABLE_DISPLAY_NAME + " [1]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "            SERVER SORTED BY [\"S.:supplier_id\"]\n" +
                 "        CLIENT MERGE SORT\n" +
@@ -252,7 +252,7 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "        CLIENT PARALLEL 4-WAY FULL SCAN OVER " + JOIN_COITEM_TABLE_DISPLAY_NAME + "\n" +
                 "        CLIENT MERGE SORT\n" +
                 "    AND\n" +
-                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
+                "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[1\\]\n" +
                 "            SERVER FILTER BY FIRST KEY ONLY\n" +
                 "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\".+.:item_id\", \".+.0:NAME\"\\]\n" +
                 "        CLIENT MERGE SORT\n" + 
@@ -262,7 +262,7 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "                CLIENT MERGE SORT\n" +
                 "    CLIENT SORTED BY \\[.*.CO_ITEM_ID, .*.CO_ITEM_NAME\\]\n" +
                 "AND\n" +
-                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
+                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[1\\]\n" +
                 "        SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\".+.:item_id\", \".+.0:NAME\"\\]\n" +
                 "    CLIENT MERGE SORT\n" + 
@@ -274,12 +274,12 @@ public class SubqueryUsingSortMergeJoinIT extends BaseHBaseManagedTimeIT {
                 "CLIENT FILTER BY \\(\\$\\d+.\\$\\d+ IS NOT NULL OR \\$\\d+.\\$\\d+ IS NOT NULL\\)",
                 
                 "SORT-MERGE-JOIN \\(SEMI\\) TABLES\n" +
-                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_CUSTOMER_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
+                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_CUSTOMER_TABLE_DISPLAY_NAME + " \\[1\\]\n" +
                 "        SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        SERVER SORTED BY \\[\"Join.idx_customer.:customer_id\"\\]\n" +
                 "    CLIENT MERGE SORT\n" +
                 "AND \\(SKIP MERGE\\)\n" +
-                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[-32768\\]\n" +
+                "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + JOIN_ITEM_TABLE_DISPLAY_NAME + " \\[1\\]\n" +
                 "        SERVER FILTER BY FIRST KEY ONLY\n" +
                 "        SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"O.customer_id\"\\]\n" +
                 "    CLIENT MERGE SORT\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
----------------------------------------------------------------------
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 fc9489d..520de85 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
@@ -125,7 +125,7 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("explain select pk2,col1 from acme where col1='f'");
         if(localIndex){
-            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_MT_BASE ['a',-32768,'f']\n"
+            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER MT_BASE ['a',1,'f']\n"
                     + "    SERVER FILTER BY FIRST KEY ONLY\n"
                     + "CLIENT MERGE SORT",QueryUtil.getExplainPlan(rs));
         } else {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
----------------------------------------------------------------------
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 37d285f..64c0d26 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
@@ -166,7 +166,6 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
 
             HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin();
             assertTrue(admin.tableExists(phoenixFullTableName));
-            assertTrue(admin.tableExists(MetaDataUtil.getLocalIndexPhysicalName(Bytes.toBytes(phoenixFullTableName))));
             assertTrue(admin.tableExists(schemaName + QueryConstants.NAME_SEPARATOR + indexName));
             assertTrue(admin.tableExists(MetaDataUtil.getViewIndexPhysicalName(Bytes.toBytes(phoenixFullTableName))));
             Properties props = new Properties();
@@ -183,7 +182,7 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
             String hbaseTableName = SchemaUtil.getPhysicalTableName(Bytes.toBytes(phoenixFullTableName), true)
                     .getNameAsString();
             assertTrue(admin.tableExists(hbaseTableName));
-            assertTrue(admin.tableExists(MetaDataUtil.getLocalIndexPhysicalName(Bytes.toBytes(hbaseTableName))));
+            assertTrue(admin.tableExists(Bytes.toBytes(hbaseTableName)));
             assertTrue(admin.tableExists(schemaName + QueryConstants.NAMESPACE_SEPARATOR + indexName));
             assertTrue(admin.tableExists(MetaDataUtil.getViewIndexPhysicalName(Bytes.toBytes(hbaseTableName))));
             i = 0;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
index 27c9e47..2565223 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
@@ -776,7 +776,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT{
         stmt.execute("create local index idx2 on t5(myreverse5(lastname_reverse))");
         query = "select k,k1,myreverse5(lastname_reverse) from t5 where myreverse5(lastname_reverse)='kcoj'";
         rs = stmt.executeQuery("explain " + query);
-        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_T5 [-32768,'kcoj']\n"
+        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER T5 [1,'kcoj']\n"
                 + "    SERVER FILTER BY FIRST KEY ONLY\n"
                 +"CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
         rs = stmt.executeQuery(query);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
----------------------------------------------------------------------
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 c87d82d..ab58840 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
@@ -523,7 +523,7 @@ public class ViewIT extends BaseViewIT {
         String queryPlan = QueryUtil.getExplainPlan(rs);
         // Assert that in either case (local & global) that index from physical table used for query on view.
         if (localIndex) {
-            assertEquals("CLIENT PARALLEL 1-WAY SKIP SCAN ON 4 KEYS OVER _LOCAL_IDX_" + fullTableName + " [-32768,1,100] - [-32768,2,109]\n" + 
+            assertEquals("CLIENT PARALLEL 1-WAY SKIP SCAN ON 4 KEYS OVER " + fullTableName + " [1,1,100] - [1,2,109]\n" + 
                     "    SERVER FILTER BY (\"S2\" = 'bas' AND \"S1\" = 'foo')\n" + 
                     "CLIENT MERGE SORT", queryPlan);
         } else {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
index 4c60ea3..ec360ad 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
@@ -196,11 +196,12 @@ public class DropMetadataIT extends BaseHBaseManagedTimeIT {
             
             // there should be a single row belonging to localIndexTableName2 
             Scan scan = new Scan();
+            scan.addFamily(QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES);
             HTable table = (HTable) conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(localIndexTablePhysicalName.getBytes());
             ResultScanner results = table.getScanner(scan);
             Result result = results.next();
             assertNotNull(result);
-            assertNotNull("localIndexTableName2 row is missing", result.getValue(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, 
+            assertNotNull("localIndexTableName2 row is missing", result.getValue(QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES, 
                 IndexUtil.getIndexColumnName(QueryConstants.DEFAULT_COLUMN_FAMILY, "V1").getBytes()));
             assertNull(results.next());
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
index 5b513d0..9cfd9ab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
@@ -150,9 +150,9 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             // verify that the query does a range scan on the index table
             ResultSet rs = stmt.executeQuery("EXPLAIN " + whereSql);
             assertEquals(
-                    localIndex ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_INDEX_TEST."
+                    localIndex ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER INDEX_TEST."
                             + dataTableName
-                            + " [-32768,'VARCHAR1_CHAR1     _A.VARCHAR1_B.CHAR1   ',3,'2015-01-02 00:00:00.000',1,420,156,800,000,1,420,156,800,000]\nCLIENT MERGE SORT"
+                            + " [1,'VARCHAR1_CHAR1     _A.VARCHAR1_B.CHAR1   ',3,'2015-01-02 00:00:00.000',1,420,156,800,000,1,420,156,800,000]\nCLIENT MERGE SORT"
                             : "CLIENT PARALLEL 1-WAY RANGE SCAN OVER INDEX_TEST.IDX ['VARCHAR1_CHAR1     _A.VARCHAR1_B.CHAR1   ',3,'2015-01-02 00:00:00.000',1,420,156,800,000,1,420,156,800,000]",
                     QueryUtil.getExplainPlan(rs));
 
@@ -172,8 +172,8 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
                     + "from "
                     + fullDataTableName;
             rs = conn.createStatement().executeQuery("EXPLAIN " + indexSelectSql);
-            assertEquals(localIndex ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullDataTableName
-                    + " [-32768]\nCLIENT MERGE SORT" : "CLIENT PARALLEL 1-WAY FULL SCAN OVER INDEX_TEST.IDX",
+            assertEquals(localIndex ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullDataTableName
+                    + " [1]\nCLIENT MERGE SORT" : "CLIENT PARALLEL 1-WAY FULL SCAN OVER INDEX_TEST.IDX",
                     QueryUtil.getExplainPlan(rs));
             rs = conn.createStatement().executeQuery(indexSelectSql);
             verifyResult(rs, 1);
@@ -477,7 +477,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
                     + " GROUP BY (int_col1+int_col2)";
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + groupBySql);
             String expectedPlan = "CLIENT PARALLEL 1-WAY "
-                    + (localIndex ? "RANGE SCAN OVER _LOCAL_IDX_" + fullDataTableName + " [-32768]"
+                    + (localIndex ? "RANGE SCAN OVER " + fullDataTableName + " [1]"
                             : "FULL SCAN OVER INDEX_TEST.IDX")
                     + "\n    SERVER FILTER BY FIRST KEY ONLY\n    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [TO_BIGINT(\"(A.INT_COL1 + B.INT_COL2)\")]" 
                     + (localIndex ? "\nCLIENT MERGE SORT" : "");
@@ -529,7 +529,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             String sql = "SELECT distinct int_col1+1 FROM " + fullDataTableName + " where int_col1+1 > 0";
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + sql);
             String expectedPlan = "CLIENT PARALLEL 1-WAY RANGE SCAN OVER "
-                    + (localIndex ? "_LOCAL_IDX_" + fullDataTableName + " [-32768,0] - [-32768,*]"
+                    + (localIndex ? fullDataTableName + " [1,0] - [1,*]"
                             : "INDEX_TEST.IDX [0] - [*]")
                     + "\n    SERVER FILTER BY FIRST KEY ONLY\n    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [TO_BIGINT(\"(A.INT_COL1 + 1)\")]"
                     + (localIndex ? "\nCLIENT MERGE SORT" : "");
@@ -582,7 +582,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             String sql = "SELECT int_col1+1 FROM " + fullDataTableName + " where int_col1+1 IN (2)";
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + sql);
             assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER "
-                    + (localIndex ? "_LOCAL_IDX_" + fullDataTableName + " [-32768,2]\n    SERVER FILTER BY FIRST KEY ONLY\nCLIENT MERGE SORT"
+                    + (localIndex ? fullDataTableName + " [1,2]\n    SERVER FILTER BY FIRST KEY ONLY\nCLIENT MERGE SORT"
                             : "INDEX_TEST.IDX [2]\n    SERVER FILTER BY FIRST KEY ONLY"), QueryUtil.getExplainPlan(rs));
             rs = conn.createStatement().executeQuery(sql);
             assertTrue(rs.next());
@@ -630,8 +630,8 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             String sql = "SELECT int_col1+1 AS foo FROM " + fullDataTableName + " ORDER BY foo";
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + sql);
             assertEquals("CLIENT PARALLEL 1-WAY "
-                    + (localIndex ? "RANGE SCAN OVER _LOCAL_IDX_" + fullDataTableName
-                            + " [-32768]\n    SERVER FILTER BY FIRST KEY ONLY\nCLIENT MERGE SORT"
+                    + (localIndex ? "RANGE SCAN OVER " + fullDataTableName
+                            + " [1]\n    SERVER FILTER BY FIRST KEY ONLY\nCLIENT MERGE SORT"
                             : "FULL SCAN OVER INDEX_TEST.IDX\n    SERVER FILTER BY FIRST KEY ONLY"),
                     QueryUtil.getExplainPlan(rs));
             rs = conn.createStatement().executeQuery(sql);
@@ -695,7 +695,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             query = "SELECT (\"V1\" || '_' || \"v2\"), k, \"V1\", \"v2\"  FROM cs WHERE (\"V1\" || '_' || \"v2\") = 'x_1'";
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_CS [-32768,'x_1']\n"
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER CS [1,'x_1']\n"
                            + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER ICS ['x_1']", QueryUtil.getExplainPlan(rs));
@@ -717,7 +717,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             query = "SELECT \"V1\", \"V1\" as foo1, (\"V1\" || '_' || \"v2\") as foo, (\"V1\" || '_' || \"v2\") as \"Foo1\", (\"V1\" || '_' || \"v2\") FROM cs ORDER BY foo";
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_CS [-32768]\nCLIENT MERGE SORT",
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER CS [1]\nCLIENT MERGE SORT",
                     QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER ICS", QueryUtil.getExplainPlan(rs));
@@ -791,8 +791,8 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             String sql = "SELECT int_col1+1, int_col2 FROM " + fullDataTableName + " WHERE int_col1+1=2";
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + sql);
             assertEquals("CLIENT PARALLEL 1-WAY "
-                    + (localIndex ? "RANGE SCAN OVER _LOCAL_IDX_" + fullDataTableName
-                            + " [-32768,2]\n    SERVER FILTER BY FIRST KEY ONLY\nCLIENT MERGE SORT" : "FULL SCAN OVER "
+                    + (localIndex ? "RANGE SCAN OVER " + fullDataTableName
+                            + " [1,2]\n    SERVER FILTER BY FIRST KEY ONLY\nCLIENT MERGE SORT" : "FULL SCAN OVER "
                             + fullDataTableName + "\n    SERVER FILTER BY (A.INT_COL1 + 1) = 2"),
                     QueryUtil.getExplainPlan(rs));
             rs = conn.createStatement().executeQuery(sql);
@@ -1163,7 +1163,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        String queryPlan = QueryUtil.getExplainPlan(rs);
 	        if (local) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_T [-32768,173]\n" + "CLIENT MERGE SORT",
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER T [1,173]\n" + "CLIENT MERGE SORT",
 	                    queryPlan);
 	        } else {
 	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [" + Short.MIN_VALUE + ",173]", queryPlan);
@@ -1182,7 +1182,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 	        query = "SELECT k1, k2, s1||'_'||s2 FROM v WHERE (s1||'_'||s2)='foo2_bar2'";
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if (local) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_T [" + (Short.MIN_VALUE + 1)
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER T [" + (2)
 	                    + ",'foo2_bar2']\n" + "    SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT",
 	                    QueryUtil.getExplainPlan(rs));
 	        } else {
@@ -1325,7 +1325,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 			rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 			if (localIndex) {
 				assertEquals(
-						"CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_T [-32768,'id:id1']\n"
+						"CLIENT PARALLEL 1-WAY RANGE SCAN OVER T [1,'id:id1']\n"
 								+ "    SERVER FILTER BY FIRST KEY ONLY\nCLIENT MERGE SORT",
 						QueryUtil.getExplainPlan(rs));
 			} else {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index 4a7e053..ff26208 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -131,7 +131,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 	        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if(localIndex) {
 	            assertEquals(
-	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + tableName + " [-32768]\n" + 
+	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [1]\n" + 
 	                "    SERVER FILTER BY FIRST KEY ONLY\n" +
 	                "CLIENT MERGE SORT",
 	                QueryUtil.getExplainPlan(rs));
@@ -488,7 +488,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             String query = "SELECT int_pk from " + fullTableName ;
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if (localIndex) {
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName +" [-32768]\n"
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName +" [1]\n"
                            + "    SERVER FILTER BY FIRST KEY ONLY\n"
                            + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
@@ -508,7 +508,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             query = "SELECT date_col from " + fullTableName + " order by date_col" ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if (localIndex) {
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName + " [-32768]\n"
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY\n"
                         + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
@@ -560,7 +560,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 	        query = "SELECT * FROM " + fullTableName;
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if(localIndex){
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
 	        } else {
 	            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
 	        }
@@ -585,7 +585,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 	        query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if(localIndex){
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" +fullTableName + " [-32768,~'1']\n" + 
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" + 
 	                    "    SERVER SORTED BY [\"V1\"]\n" + 
 	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
 	        } else {
@@ -640,7 +640,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 	        query = "SELECT a.* FROM " + fullTableName;
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if(localIndex) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
 	        } else {
 	            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
 	        }
@@ -779,7 +779,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 	        query = "SELECT * FROM " + testTable;
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if (localIndex) {
-	            assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER _LOCAL_IDX_" + testTable+" [-32768]\n"
+	            assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [1]\n"
 	                       + "    SERVER FILTER BY FIRST KEY ONLY\n"
 	                       + "CLIENT MERGE SORT",
 	                QueryUtil.getExplainPlan(rs));
@@ -838,7 +838,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             query = "SELECT * FROM " + fullTableName + " WHERE \"v2\" = '1'";
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName + " [-32768,'1']\n"
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1,'1']\n"
                            + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullIndexName + " ['1']", QueryUtil.getExplainPlan(rs));
@@ -857,7 +857,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             query = "SELECT \"V1\", \"V1\" as foo1, \"v2\" as foo, \"v2\" as \"Foo1\", \"v2\" FROM " + fullTableName + " ORDER BY foo";
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName + " [-32768]\nCLIENT MERGE SORT",
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\nCLIENT MERGE SORT",
                     QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER "+fullIndexName, QueryUtil.getExplainPlan(rs));
@@ -927,7 +927,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + fullTableName ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
             }


[3/5] phoenix git commit: PHOENIX-1734 Local index improvements(Rajeshbabu)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java
deleted file mode 100644
index d835ce9..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java
+++ /dev/null
@@ -1,986 +0,0 @@
-/**
- *
- * 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.hadoop.hbase.regionserver;
-
-import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REQUEST_REGION_SPLIT;
-import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REGION_SPLIT;
-import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REGION_SPLITTING;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.RegionTransition;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.io.Reference;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.CancelableProgressable;
-import org.apache.phoenix.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.HasThread;
-import org.apache.hadoop.hbase.util.PairOfSameType;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
-import org.apache.zookeeper.data.Stat;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-@InterfaceAudience.Private
-public class IndexSplitTransaction extends SplitTransactionImpl { // FIXME: Extends private type
-  private static final Log LOG = LogFactory.getLog(IndexSplitTransaction.class);
-
-  /*
-   * Region to split
-   */
-  private final HRegion parent;
-  private HRegionInfo hri_a;
-  private HRegionInfo hri_b;
-  private long fileSplitTimeout = 30000;
-  private int znodeVersion = -1;
-
-  /*
-   * Row to split around
-   */
-  private final byte [] splitrow;
-
-  /**
-   * Types to add to the transaction journal.
-   * Each enum is a step in the split transaction. Used to figure how much
-   * we need to rollback.
-   */
-  enum JournalEntry {
-    /**
-     * Set region as in transition, set it into SPLITTING state.
-     */
-    SET_SPLITTING_IN_ZK,
-    /**
-     * We created the temporary split data directory.
-     */
-    CREATE_SPLIT_DIR,
-    /**
-     * Closed the parent region.
-     */
-    CLOSED_PARENT_REGION,
-    /**
-     * The parent has been taken out of the server's online regions list.
-     */
-    OFFLINED_PARENT,
-    /**
-     * Started in on creation of the first daughter region.
-     */
-    STARTED_REGION_A_CREATION,
-    /**
-     * Started in on the creation of the second daughter region.
-     */
-    STARTED_REGION_B_CREATION,
-    /**
-     * Point of no return.
-     * If we got here, then transaction is not recoverable other than by
-     * crashing out the regionserver.
-     */
-    PONR
-  }
-
-  /*
-   * Journal of how far the split transaction has progressed.
-   */
-  private final List<JournalEntry> journal = new ArrayList<JournalEntry>();
-
-  /**
-   * Constructor
-   * @param r Region to split
-   * @param splitrow Row to split around
-   */
-  public IndexSplitTransaction(final Region r, final byte [] splitrow) {
-    super(r , splitrow);
-    this.parent = (HRegion)r;
-    this.splitrow = splitrow;
-  }
-
-  /**
-   * Does checks on split inputs.
-   * @return <code>true</code> if the region is splittable else
-   * <code>false</code> if it is not (e.g. its already closed, etc.).
-   */
-  @Override
-  public boolean prepare() {
-    if (!this.parent.isSplittable()) return false;
-    // Split key can be null if this region is unsplittable; i.e. has refs.
-    if (this.splitrow == null) return false;
-    HRegionInfo hri = this.parent.getRegionInfo();
-    parent.prepareToSplit();
-    // Check splitrow.
-    byte [] startKey = hri.getStartKey();
-    byte [] endKey = hri.getEndKey();
-    if (Bytes.equals(startKey, splitrow) ||
-        !this.parent.getRegionInfo().containsRow(splitrow)) {
-      LOG.info("Split row is not inside region key range or is equal to " +
-          "startkey: " + Bytes.toStringBinary(this.splitrow));
-      return false;
-    }
-    long rid = getDaughterRegionIdTimestamp(hri);
-    this.hri_a = new HRegionInfo(hri.getTable(), startKey, this.splitrow, false, rid);
-    this.hri_b = new HRegionInfo(hri.getTable(), this.splitrow, endKey, false, rid);
-    return true;
-  }
-
-  /**
-   * Calculate daughter regionid to use.
-   * @param hri Parent {@link HRegionInfo}
-   * @return Daughter region id (timestamp) to use.
-   */
-  private static long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
-    long rid = EnvironmentEdgeManager.currentTimeMillis();
-    // Regionid is timestamp.  Can't be less than that of parent else will insert
-    // at wrong location in hbase:meta (See HBASE-710).
-    if (rid < hri.getRegionId()) {
-      LOG.warn("Clock skew; parent regions id is " + hri.getRegionId() +
-        " but current time here is " + rid);
-      rid = hri.getRegionId() + 1;
-    }
-    return rid;
-  }
-
-  private static IOException closedByOtherException = new IOException(
-      "Failed to close region: already closed by another thread");
-
-  /**
-   * Prepare the regions and region files.
-   * @param server Hosting server instance.  Can be null when testing (won't try
-   * and update in zk if a null server)
-   * @param services Used to online/offline regions.
-   * @throws IOException If thrown, transaction failed.
-   *    Call {@link #rollback(Server, RegionServerServices)}
-   * @return Regions created
-   */
-  @Override
-  /* package */PairOfSameType<Region> createDaughters(final Server server,
-      final RegionServerServices services) throws IOException {
-    LOG.info("Starting split of region " + this.parent);
-    if ((server != null && server.isStopped()) ||
-        (services != null && services.isStopping())) {
-      throw new IOException("Server is stopped or stopping");
-    }
-    assert !this.parent.lock.writeLock().isHeldByCurrentThread():
-      "Unsafe to hold write lock while performing RPCs";
-
-    // Coprocessor callback
-    if (this.parent.getCoprocessorHost() != null) {
-      this.parent.getCoprocessorHost().preSplit();
-    }
-
-    // Coprocessor callback
-    if (this.parent.getCoprocessorHost() != null) {
-      this.parent.getCoprocessorHost().preSplit(this.splitrow);
-    }
-
-    // If true, no cluster to write meta edits to or to update znodes in.
-    boolean testing = server == null? true:
-        server.getConfiguration().getBoolean("hbase.testing.nocluster", false);
-    this.fileSplitTimeout = testing ? this.fileSplitTimeout :
-        server.getConfiguration().getLong("hbase.regionserver.fileSplitTimeout",
-          this.fileSplitTimeout);
-
-    PairOfSameType<Region> daughterRegions = stepsBeforePONR(server, services, testing);
-
-    List<Mutation> metaEntries = new ArrayList<Mutation>();
-    if (this.parent.getCoprocessorHost() != null) {
-      if (this.parent.getCoprocessorHost().
-          preSplitBeforePONR(this.splitrow, metaEntries)) {
-        throw new IOException("Coprocessor bypassing region "
-            + this.parent.getRegionInfo().getRegionNameAsString() + " split.");
-      }
-      try {
-        for (Mutation p : metaEntries) {
-          HRegionInfo.parseRegionName(p.getRow());
-        }
-      } catch (IOException e) {
-        LOG.error("Row key of mutation from coprossor is not parsable as region name."
-            + "Mutations from coprocessor should only for hbase:meta table.");
-        throw e;
-      }
-    }
-
-    // This is the point of no return.  Adding subsequent edits to .META. as we
-    // do below when we do the daughter opens adding each to .META. can fail in
-    // various interesting ways the most interesting of which is a timeout
-    // BUT the edits all go through (See HBASE-3872).  IF we reach the PONR
-    // then subsequent failures need to crash out this regionserver; the
-    // server shutdown processing should be able to fix-up the incomplete split.
-    // The offlined parent will have the daughters as extra columns.  If
-    // we leave the daughter regions in place and do not remove them when we
-    // crash out, then they will have their references to the parent in place
-    // still and the server shutdown fixup of .META. will point to these
-    // regions.
-    // We should add PONR JournalEntry before offlineParentInMeta,so even if
-    // OfflineParentInMeta timeout,this will cause regionserver exit,and then
-    // master ServerShutdownHandler will fix daughter & avoid data loss. (See
-    // HBase-4562).
-    this.journal.add(JournalEntry.PONR);
-
-    // Edit parent in meta.  Offlines parent region and adds splita and splitb
-    // as an atomic update. See HBASE-7721. This update to META makes the region
-    // will determine whether the region is split or not in case of failures.
-    // If it is successful, master will roll-forward, if not, master will rollback
-    // and assign the parent region.
-    if (!testing) {
-      if (metaEntries == null || metaEntries.isEmpty()) {
-        MetaTableAccessor.splitRegion(server.getConnection(), parent.getRegionInfo(),
-                daughterRegions.getFirst().getRegionInfo(),
-                daughterRegions.getSecond().getRegionInfo(), server.getServerName(),
-                parent.getTableDesc().getRegionReplication());
-      } else {
-        offlineParentInMetaAndputMetaEntries(server.getConnection(),
-          parent.getRegionInfo(), daughterRegions.getFirst().getRegionInfo(), daughterRegions
-              .getSecond().getRegionInfo(), server.getServerName(), metaEntries,
-              parent.getTableDesc().getRegionReplication());
-      }
-    }
-    return daughterRegions;
-  }
-
-  @Override
-  public PairOfSameType<Region> stepsBeforePONR(final Server server,
-      final RegionServerServices services, boolean testing) throws IOException {
-    // Set ephemeral SPLITTING znode up in zk.  Mocked servers sometimes don't
-    // have zookeeper so don't do zk stuff if server or zookeeper is null
-    if (server != null && server.getZooKeeper() != null) {
-      try {
-        createNodeSplitting(server.getZooKeeper(),
-          parent.getRegionInfo(), server.getServerName(), hri_a, hri_b);
-      } catch (KeeperException e) {
-        throw new IOException("Failed creating PENDING_SPLIT znode on " +
-          this.parent.getRegionInfo().getRegionNameAsString(), e);
-      }
-    }
-    this.journal.add(JournalEntry.SET_SPLITTING_IN_ZK);
-    if (server != null && server.getZooKeeper() != null) {
-      // After creating the split node, wait for master to transition it
-      // from PENDING_SPLIT to SPLITTING so that we can move on. We want master
-      // knows about it and won't transition any region which is splitting.
-      znodeVersion = getZKNode(server, services);
-    }
-
-    this.parent.getRegionFileSystem().createSplitsDir();
-    this.journal.add(JournalEntry.CREATE_SPLIT_DIR);
-
-    Map<byte[], List<StoreFile>> hstoreFilesToSplit = null;
-    Exception exceptionToThrow = null;
-    try{
-      hstoreFilesToSplit = this.parent.close(false);
-    } catch (Exception e) {
-      exceptionToThrow = e;
-    }
-    if (exceptionToThrow == null && hstoreFilesToSplit == null) {
-      // The region was closed by a concurrent thread.  We can't continue
-      // with the split, instead we must just abandon the split.  If we
-      // reopen or split this could cause problems because the region has
-      // probably already been moved to a different server, or is in the
-      // process of moving to a different server.
-      exceptionToThrow = closedByOtherException;
-    }
-    if (exceptionToThrow != closedByOtherException) {
-      this.journal.add(JournalEntry.CLOSED_PARENT_REGION);
-    }
-    if (exceptionToThrow != null) {
-      if (exceptionToThrow instanceof IOException) throw (IOException)exceptionToThrow;
-      throw new IOException(exceptionToThrow);
-    }
-    if (!testing) {
-      services.removeFromOnlineRegions(this.parent, null);
-    }
-    this.journal.add(JournalEntry.OFFLINED_PARENT);
-
-    // TODO: If splitStoreFiles were multithreaded would we complete steps in
-    // less elapsed time?  St.Ack 20100920
-    //
-    // splitStoreFiles creates daughter region dirs under the parent splits dir
-    // Nothing to unroll here if failure -- clean up of CREATE_SPLIT_DIR will
-    // clean this up.
-    splitStoreFiles(hstoreFilesToSplit);
-
-    // Log to the journal that we are creating region A, the first daughter
-    // region.  We could fail halfway through.  If we do, we could have left
-    // stuff in fs that needs cleanup -- a storefile or two.  Thats why we
-    // add entry to journal BEFORE rather than AFTER the change.
-    this.journal.add(JournalEntry.STARTED_REGION_A_CREATION);
-    Region a = this.parent.createDaughterRegionFromSplits(this.hri_a);
-
-    // Ditto
-    this.journal.add(JournalEntry.STARTED_REGION_B_CREATION);
-    Region b = this.parent.createDaughterRegionFromSplits(this.hri_b);
-    return new PairOfSameType<Region>(a, b);
-  }
-
-  /**
-   * Perform time consuming opening of the daughter regions.
-   * @param server Hosting server instance.  Can be null when testing (won't try
-   * and update in zk if a null server)
-   * @param services Used to online/offline regions.
-   * @param a first daughter region
-   * @param a second daughter region
-   * @throws IOException If thrown, transaction failed.
-   *          Call {@link #rollback(Server, RegionServerServices)}
-   */
-  @Override
-  /* package */void openDaughters(final Server server,
-      final RegionServerServices services, Region a, Region b)
-      throws IOException {
-    boolean stopped = server != null && server.isStopped();
-    boolean stopping = services != null && services.isStopping();
-    // TODO: Is this check needed here?
-    if (stopped || stopping) {
-      LOG.info("Not opening daughters " +
-          b.getRegionInfo().getRegionNameAsString() +
-          " and " +
-          a.getRegionInfo().getRegionNameAsString() +
-          " because stopping=" + stopping + ", stopped=" + stopped);
-    } else {
-      // Open daughters in parallel.
-      DaughterOpener aOpener = new DaughterOpener(server, (HRegion)a);
-      DaughterOpener bOpener = new DaughterOpener(server, (HRegion)b);
-      aOpener.start();
-      bOpener.start();
-      try {
-        aOpener.join();
-        bOpener.join();
-      } catch (InterruptedException e) {
-        throw (InterruptedIOException)new InterruptedIOException().initCause(e);
-      }
-      if (aOpener.getException() != null) {
-        throw new IOException("Failed " +
-          aOpener.getName(), aOpener.getException());
-      }
-      if (bOpener.getException() != null) {
-        throw new IOException("Failed " +
-          bOpener.getName(), bOpener.getException());
-      }
-      if (services != null) {
-        try {
-          // add 2nd daughter first (see HBASE-4335)
-          services.postOpenDeployTasks(b);
-          // Should add it to OnlineRegions
-          services.addToOnlineRegions(b);
-          services.postOpenDeployTasks(a);
-          services.addToOnlineRegions(a);
-        } catch (KeeperException ke) {
-          throw new IOException(ke);
-        }
-      }
-    }
-  }
-
-  /**
-   * Finish off split transaction, transition the zknode
-   * @param server Hosting server instance.  Can be null when testing (won't try
-   * and update in zk if a null server)
-   * @param services Used to online/offline regions.
-   * @param a first daughter region
-   * @param a second daughter region
-   * @throws IOException If thrown, transaction failed.
-   *          Call {@link #rollback(Server, RegionServerServices)}
-   */
-  /* package */void transitionZKNode(final Server server,
-      final RegionServerServices services, Region a, Region b)
-      throws IOException {
-    // Tell master about split by updating zk.  If we fail, abort.
-    if (server != null && server.getZooKeeper() != null) {
-      try {
-        this.znodeVersion = transitionSplittingNode(server.getZooKeeper(),
-          parent.getRegionInfo(), a.getRegionInfo(), b.getRegionInfo(),
-          server.getServerName(), this.znodeVersion,
-          RS_ZK_REGION_SPLITTING, RS_ZK_REGION_SPLIT);
-
-        int spins = 0;
-        // Now wait for the master to process the split. We know it's done
-        // when the znode is deleted. The reason we keep tickling the znode is
-        // that it's possible for the master to miss an event.
-        do {
-          if (spins % 10 == 0) {
-            LOG.debug("Still waiting on the master to process the split for " +
-                this.parent.getRegionInfo().getEncodedName());
-          }
-          Thread.sleep(100);
-          // When this returns -1 it means the znode doesn't exist
-          this.znodeVersion = transitionSplittingNode(server.getZooKeeper(),
-            parent.getRegionInfo(), a.getRegionInfo(), b.getRegionInfo(),
-            server.getServerName(), this.znodeVersion,
-            RS_ZK_REGION_SPLIT, RS_ZK_REGION_SPLIT);
-          spins++;
-        } while (this.znodeVersion != -1 && !server.isStopped()
-            && !services.isStopping());
-      } catch (Exception e) {
-        if (e instanceof InterruptedException) {
-          Thread.currentThread().interrupt();
-        }
-        throw new IOException("Failed telling master about split", e);
-      }
-    }
-
-    // Coprocessor callback
-    if (this.parent.getCoprocessorHost() != null) {
-      this.parent.getCoprocessorHost().postSplit(a,b);
-    }
-
-    // Leaving here, the splitdir with its dross will be in place but since the
-    // split was successful, just leave it; it'll be cleaned when parent is
-    // deleted and cleaned up.
-  }
-
-  /**
-   * Wait for the splitting node to be transitioned from pending_split
-   * to splitting by master. That's how we are sure master has processed
-   * the event and is good with us to move on. If we don't get any update,
-   * we periodically transition the node so that master gets the callback.
-   * If the node is removed or is not in pending_split state any more,
-   * we abort the split.
-   */
-  private int getZKNode(final Server server,
-      final RegionServerServices services) throws IOException {
-    // Wait for the master to process the pending_split.
-    try {
-      int spins = 0;
-      Stat stat = new Stat();
-      ZooKeeperWatcher zkw = server.getZooKeeper();
-      ServerName expectedServer = server.getServerName();
-      String node = parent.getRegionInfo().getEncodedName();
-      while (!(server.isStopped() || services.isStopping())) {
-        if (spins % 5 == 0) {
-          LOG.debug("Still waiting for master to process "
-            + "the pending_split for " + node);
-          transitionSplittingNode(zkw, parent.getRegionInfo(),
-            hri_a, hri_b, expectedServer, -1, RS_ZK_REQUEST_REGION_SPLIT,
-            RS_ZK_REQUEST_REGION_SPLIT);
-        }
-        Thread.sleep(100);
-        spins++;
-        byte [] data = ZKAssign.getDataNoWatch(zkw, node, stat);
-        if (data == null) {
-          throw new IOException("Data is null, splitting node "
-            + node + " no longer exists");
-        }
-        RegionTransition rt = RegionTransition.parseFrom(data);
-        EventType et = rt.getEventType();
-        if (et == RS_ZK_REGION_SPLITTING) {
-          ServerName serverName = rt.getServerName();
-          if (!serverName.equals(expectedServer)) {
-            throw new IOException("Splitting node " + node + " is for "
-              + serverName + ", not us " + expectedServer);
-          }
-          byte [] payloadOfSplitting = rt.getPayload();
-          List<HRegionInfo> splittingRegions = HRegionInfo.parseDelimitedFrom(
-            payloadOfSplitting, 0, payloadOfSplitting.length);
-          assert splittingRegions.size() == 2;
-          HRegionInfo a = splittingRegions.get(0);
-          HRegionInfo b = splittingRegions.get(1);
-          if (!(hri_a.equals(a) && hri_b.equals(b))) {
-            throw new IOException("Splitting node " + node + " is for " + a + ", "
-              + b + ", not expected daughters: " + hri_a + ", " + hri_b);
-          }
-          // Master has processed it.
-          return stat.getVersion();
-        }
-        if (et != RS_ZK_REQUEST_REGION_SPLIT) {
-          throw new IOException("Splitting node " + node
-            + " moved out of splitting to " + et);
-        }
-      }
-      // Server is stopping/stopped
-      throw new IOException("Server is "
-        + (services.isStopping() ? "stopping" : "stopped"));
-    } catch (Exception e) {
-      if (e instanceof InterruptedException) {
-        Thread.currentThread().interrupt();
-      }
-      throw new IOException("Failed getting SPLITTING znode on "
-        + parent.getRegionInfo().getRegionNameAsString(), e);
-    }
-  }
-
-  /**
-   * Run the transaction.
-   * @param server Hosting server instance.  Can be null when testing (won't try
-   * and update in zk if a null server)
-   * @param services Used to online/offline regions.
-   * @throws IOException If thrown, transaction failed.
-   *          Call {@link #rollback(Server, RegionServerServices)}
-   * @return Regions created
-   * @throws IOException
-   * @see #rollback(Server, RegionServerServices)
-   */
-  @Override
-  public PairOfSameType<Region> execute(final Server server,
-      final RegionServerServices services)
-  throws IOException {
-    PairOfSameType<Region> regions = createDaughters(server, services);
-    if (this.parent.getCoprocessorHost() != null) {
-      this.parent.getCoprocessorHost().preSplitAfterPONR();
-    }
-    return stepsAfterPONR(server, services, regions);
-  }
-
-  @Override
-  public PairOfSameType<Region> stepsAfterPONR(final Server server,
-      final RegionServerServices services, PairOfSameType<Region> regions)
-      throws IOException {
-    openDaughters(server, services, regions.getFirst(), regions.getSecond());
-    transitionZKNode(server, services, regions.getFirst(), regions.getSecond());
-    return regions;
-  }
-
-  private void offlineParentInMetaAndputMetaEntries(Connection conn,
-      HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB,
-      ServerName serverName, List<Mutation> metaEntries, int regionReplication) throws IOException {
-    List<Mutation> mutations = metaEntries;
-    HRegionInfo copyOfParent = new HRegionInfo(parent);
-    copyOfParent.setOffline(true);
-    copyOfParent.setSplit(true);
-
-    //Put for parent
-    Put putParent = MetaTableAccessor.makePutFromRegionInfo(copyOfParent);
-    MetaTableAccessor.addDaughtersToPut(putParent, splitA, splitB);
-    mutations.add(putParent);
-
-    //Puts for daughters
-    Put putA = MetaTableAccessor.makePutFromRegionInfo(splitA);
-    Put putB = MetaTableAccessor.makePutFromRegionInfo(splitB);
-
-    addLocation(putA, serverName, 1); //these are new regions, openSeqNum = 1 is fine.
-    addLocation(putB, serverName, 1);
-    mutations.add(putA);
-    mutations.add(putB);
-
-    // Add empty locations for region replicas of daughters so that number of replicas can be
-    // cached whenever the primary region is looked up from meta
-    for (int i = 1; i < regionReplication; i++) {
-      addEmptyLocation(putA, i);
-      addEmptyLocation(putB, i);
-    }
-
-    MetaTableAccessor.mutateMetaTable(conn, mutations);
-  }
-
-  @Override
-  public Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
-    p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
-      Bytes.toBytes(sn.getHostAndPort()));
-    p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
-      Bytes.toBytes(sn.getStartcode()));
-    p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SEQNUM_QUALIFIER,
-        Bytes.toBytes(openSeqNum));
-    return p;
-  }
-
-  private static Put addEmptyLocation(final Put p, int replicaId){
-    p.addImmutable(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(replicaId), null);
-    p.addImmutable(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(replicaId), null);
-    p.addImmutable(HConstants.CATALOG_FAMILY, MetaTableAccessor.getSeqNumColumn(replicaId), null);
-    return p;
-  }
-
-  /*
-   * Open daughter region in its own thread.
-   * If we fail, abort this hosting server.
-   */
-  class DaughterOpener extends HasThread {
-    private final Server server;
-    private final HRegion r;
-    private Throwable t = null;
-
-    DaughterOpener(final Server s, final HRegion r) {
-      super((s == null? "null-services": s.getServerName()) +
-        "-daughterOpener=" + r.getRegionInfo().getEncodedName());
-      setDaemon(true);
-      this.server = s;
-      this.r = r;
-    }
-
-    /**
-     * @return Null if open succeeded else exception that causes us fail open.
-     * Call it after this thread exits else you may get wrong view on result.
-     */
-    Throwable getException() {
-      return this.t;
-    }
-
-    @Override
-    public void run() {
-      try {
-        openDaughterRegion(this.server, r);
-      } catch (Throwable t) {
-        this.t = t;
-      }
-    }
-  }
-
-  /**
-   * Open daughter regions, add them to online list and update meta.
-   * @param server
-   * @param daughter
-   * @throws IOException
-   * @throws KeeperException
-   */
-  @Override
-  void openDaughterRegion(final Server server, final HRegion daughter)
-  throws IOException, KeeperException {
-    HRegionInfo hri = daughter.getRegionInfo();
-    LoggingProgressable reporter = server == null ? null
-        : new LoggingProgressable(hri, server.getConfiguration().getLong(
-            "hbase.regionserver.split.daughter.open.log.interval", 10000));
-    daughter.openHRegion(reporter);
-  }
-
-  static class LoggingProgressable implements CancelableProgressable {
-    private final HRegionInfo hri;
-    private long lastLog = -1;
-    private final long interval;
-
-    LoggingProgressable(final HRegionInfo hri, final long interval) {
-      this.hri = hri;
-      this.interval = interval;
-    }
-
-    @Override
-    public boolean progress() {
-      long now = System.currentTimeMillis();
-      if (now - lastLog > this.interval) {
-        LOG.info("Opening " + this.hri.getRegionNameAsString());
-        this.lastLog = now;
-      }
-      return true;
-    }
-  }
-
-  private void splitStoreFiles(final Map<byte[], List<StoreFile>> hstoreFilesToSplit)
-      throws IOException {
-    if (hstoreFilesToSplit == null) {
-      // Could be null because close didn't succeed -- for now consider it fatal
-      throw new IOException("Close returned empty list of StoreFiles");
-    }
-    // The following code sets up a thread pool executor with as many slots as
-    // there's files to split. It then fires up everything, waits for
-    // completion and finally checks for any exception
-    int nbFiles = hstoreFilesToSplit.size();
-    if (nbFiles == 0) {
-      // no file needs to be splitted.
-      return;
-    }
-    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
-    builder.setNameFormat("StoreFileSplitter-%1$d");
-    ThreadFactory factory = builder.build();
-    ThreadPoolExecutor threadPool =
-      (ThreadPoolExecutor) Executors.newFixedThreadPool(nbFiles, factory);
-    List<Future<Void>> futures = new ArrayList<Future<Void>>(nbFiles);
-
-    // Split each store file.
-    for (Map.Entry<byte[], List<StoreFile>> entry: hstoreFilesToSplit.entrySet()) {
-      for (StoreFile sf: entry.getValue()) {
-        StoreFileSplitter sfs = new StoreFileSplitter(entry.getKey(), sf);
-        futures.add(threadPool.submit(sfs));
-      }
-    }
-    // Shutdown the pool
-    threadPool.shutdown();
-
-    // Wait for all the tasks to finish
-    try {
-      boolean stillRunning = !threadPool.awaitTermination(
-          this.fileSplitTimeout, TimeUnit.MILLISECONDS);
-      if (stillRunning) {
-        threadPool.shutdownNow();
-        // wait for the thread to shutdown completely.
-        while (!threadPool.isTerminated()) {
-          Thread.sleep(50);
-        }
-        throw new IOException("Took too long to split the" +
-            " files and create the references, aborting split");
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      throw (InterruptedIOException)new InterruptedIOException().initCause(e);
-    }
-
-    // Look for any exception
-    for (Future<Void> future: futures) {
-      try {
-        future.get();
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw (InterruptedIOException)new InterruptedIOException().initCause(e);
-      } catch (ExecutionException e) {
-        throw new IOException(e);
-      }
-    }
-  }
-
-  /**
-   * Utility class used to do the file splitting / reference writing
-   * in parallel instead of sequentially.
-   */
-  class StoreFileSplitter implements Callable<Void> {
-    private final byte[] family;
-    private final StoreFile sf;
-
-    /**
-     * Constructor that takes what it needs to split
-     * @param family Family that contains the store file
-     * @param sf which file
-     */
-    public StoreFileSplitter(final byte[] family, final StoreFile sf) {
-      this.sf = sf;
-      this.family = family;
-    }
-
-    @Override
-    public Void call() throws IOException {
-      splitStoreFile(family, sf);
-      return null;
-    }
-  }
-
-    private void splitStoreFile(final byte[] family, final StoreFile sf) throws IOException {
-        HRegionFileSystem fs = this.parent.getRegionFileSystem();
-        String familyName = Bytes.toString(family);
-        splitStoreFile(this.hri_a, familyName, sf, this.splitrow, false, fs);
-        splitStoreFile(this.hri_b, familyName, sf, this.splitrow, true, fs);
-    }
-
-    private Path splitStoreFile(HRegionInfo hri, String familyName, StoreFile f, byte[] splitRow,
-            boolean top, HRegionFileSystem fs) throws IOException {
-        f.closeReader(true);
-        Path splitDir =
-                new Path(fs.getSplitsDir(hri), familyName);
-        // A reference to the bottom half of the hsf store file.
-        Reference r =
-                top ? Reference.createTopReference(splitRow) : Reference
-                        .createBottomReference(splitRow);
-        // Add the referred-to regions name as a dot separated suffix.
-        // See REF_NAME_REGEX regex above. The referred-to regions name is
-        // up in the path of the passed in <code>f</code> -- parentdir is family,
-        // then the directory above is the region name.
-        String parentRegionName = this.parent.getRegionInfo().getEncodedName();
-        // Write reference with same file id only with the other region name as
-        // suffix and into the new region location (under same family).
-        Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
-        return r.write(fs.getFileSystem(), p);
-    }
-
-  /**
-   * @param server Hosting server instance (May be null when testing).
-   * @param services
-   * @throws IOException If thrown, rollback failed.  Take drastic action.
-   * @return True if we successfully rolled back, false if we got to the point
-   * of no return and so now need to abort the server to minimize damage.
-   */
-  @Override
-  @SuppressWarnings("deprecation")
-  public boolean rollback(final Server server, final RegionServerServices services)
-  throws IOException {
-    // Coprocessor callback
-    if (this.parent.getCoprocessorHost() != null) {
-      this.parent.getCoprocessorHost().preRollBackSplit();
-    }
-
-    boolean result = true;
-    ListIterator<JournalEntry> iterator =
-      this.journal.listIterator(this.journal.size());
-    // Iterate in reverse.
-    while (iterator.hasPrevious()) {
-      JournalEntry je = iterator.previous();
-      switch(je) {
-
-      case SET_SPLITTING_IN_ZK:
-        if (server != null && server.getZooKeeper() != null) {
-          cleanZK(server, this.parent.getRegionInfo());
-        }
-        break;
-
-      case CREATE_SPLIT_DIR:
-        this.parent.writestate.writesEnabled = true;
-        this.parent.getRegionFileSystem().cleanupSplitsDir();
-        break;
-
-      case CLOSED_PARENT_REGION:
-        try {
-          // So, this returns a seqid but if we just closed and then reopened, we
-          // should be ok. On close, we flushed using sequenceid obtained from
-          // hosting regionserver so no need to propagate the sequenceid returned
-          // out of initialize below up into regionserver as we normally do.
-          // TODO: Verify.
-          this.parent.initialize();
-        } catch (IOException e) {
-          LOG.error("Failed rollbacking CLOSED_PARENT_REGION of region " +
-            this.parent.getRegionInfo().getRegionNameAsString(), e);
-          throw new RuntimeException(e);
-        }
-        break;
-
-      case STARTED_REGION_A_CREATION:
-        this.parent.getRegionFileSystem().cleanupDaughterRegion(this.hri_a);
-        break;
-
-      case STARTED_REGION_B_CREATION:
-        this.parent.getRegionFileSystem().cleanupDaughterRegion(this.hri_b);
-        break;
-
-      case OFFLINED_PARENT:
-        if (services != null) services.addToOnlineRegions(this.parent);
-        break;
-
-      case PONR:
-        // We got to the point-of-no-return so we need to just abort. Return
-        // immediately.  Do not clean up created daughter regions.  They need
-        // to be in place so we don't delete the parent region mistakenly.
-        // See HBASE-3872.
-        return false;
-
-      default:
-        throw new RuntimeException("Unhandled journal entry: " + je);
-      }
-    }
-    // Coprocessor callback
-    if (this.parent.getCoprocessorHost() != null) {
-      this.parent.getCoprocessorHost().postRollBackSplit();
-    }
-    return result;
-  }
-
-  @Override
-  HRegionInfo getFirstDaughter() {
-    return hri_a;
-  }
-
-  @Override
-  HRegionInfo getSecondDaughter() {
-    return hri_b;
-  }
-
-  private static void cleanZK(final Server server, final HRegionInfo hri) {
-    try {
-      // Only delete if its in expected state; could have been hijacked.
-      if (!ZKAssign.deleteNode(server.getZooKeeper(), hri.getEncodedName(),
-          RS_ZK_REQUEST_REGION_SPLIT, server.getServerName())) {
-        ZKAssign.deleteNode(server.getZooKeeper(), hri.getEncodedName(),
-          RS_ZK_REGION_SPLITTING, server.getServerName());
-      }
-    } catch (KeeperException.NoNodeException e) {
-      LOG.info("Failed cleanup zk node of " + hri.getRegionNameAsString(), e);
-    } catch (KeeperException e) {
-      server.abort("Failed cleanup of " + hri.getRegionNameAsString(), e);
-    }
-  }
-
-  /**
-   * Creates a new ephemeral node in the PENDING_SPLIT state for the specified region.
-   * Create it ephemeral in case regionserver dies mid-split.
-   *
-   * <p>Does not transition nodes from other states.  If a node already exists
-   * for this region, a {@link NodeExistsException} will be thrown.
-   *
-   * @param zkw zk reference
-   * @param region region to be created as offline
-   * @param serverName server event originates from
-   * @throws KeeperException
-   * @throws IOException
-   */
-  public static void createNodeSplitting(final ZooKeeperWatcher zkw, final HRegionInfo region,
-      final ServerName serverName, final HRegionInfo a,
-      final HRegionInfo b) throws KeeperException, IOException {
-    LOG.debug(zkw.prefix("Creating ephemeral node for " +
-      region.getEncodedName() + " in PENDING_SPLIT state"));
-    byte [] payload = HRegionInfo.toDelimitedByteArray(a, b);
-    RegionTransition rt = RegionTransition.createRegionTransition(
-      RS_ZK_REQUEST_REGION_SPLIT, region.getRegionName(), serverName, payload);
-    String node = ZKAssign.getNodeName(zkw, region.getEncodedName());
-    if (!ZKUtil.createEphemeralNodeAndWatch(zkw, node, rt.toByteArray())) {
-      throw new IOException("Failed create of ephemeral " + node);
-    }
-  }
-
-  /**
-   * Transitions an existing ephemeral node for the specified region which is
-   * currently in the begin state to be in the end state. Master cleans up the
-   * final SPLIT znode when it reads it (or if we crash, zk will clean it up).
-   *
-   * <p>Does not transition nodes from other states. If for some reason the
-   * node could not be transitioned, the method returns -1. If the transition
-   * is successful, the version of the node after transition is returned.
-   *
-   * <p>This method can fail and return false for three different reasons:
-   * <ul><li>Node for this region does not exist</li>
-   * <li>Node for this region is not in the begin state</li>
-   * <li>After verifying the begin state, update fails because of wrong version
-   * (this should never actually happen since an RS only does this transition
-   * following a transition to the begin state. If two RS are conflicting, one would
-   * fail the original transition to the begin state and not this transition)</li>
-   * </ul>
-   *
-   * <p>Does not set any watches.
-   *
-   * <p>This method should only be used by a RegionServer when splitting a region.
-   *
-   * @param zkw zk reference
-   * @param parent region to be transitioned to opened
-   * @param a Daughter a of split
-   * @param b Daughter b of split
-   * @param serverName server event originates from
-   * @param znodeVersion expected version of data before modification
-   * @param beginState the expected current state the znode should be
-   * @param endState the state to be transition to
-   * @return version of node after transition, -1 if unsuccessful transition
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws IOException
-   */
-  public static int transitionSplittingNode(ZooKeeperWatcher zkw,
-      HRegionInfo parent, HRegionInfo a, HRegionInfo b, ServerName serverName,
-      final int znodeVersion, final EventType beginState,
-      final EventType endState) throws KeeperException, IOException {
-    byte [] payload = HRegionInfo.toDelimitedByteArray(a, b);
-    return ZKAssign.transitionNode(zkw, parent, serverName,
-      beginState, endState, znodeVersion, payload);
-  }
-
-  public HRegion getParent() {
-    return this.parent;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java
deleted file mode 100644
index e361343..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * 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.hadoop.hbase.regionserver;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.BaseRegionServerObserver;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
-import org.apache.phoenix.schema.types.PBoolean;
-import org.apache.phoenix.util.IndexUtil;
-import org.apache.phoenix.util.MetaDataUtil;
-import org.apache.phoenix.util.SchemaUtil;
-
-public class LocalIndexMerger extends BaseRegionServerObserver {
-
-    private static final Log LOG = LogFactory.getLog(LocalIndexMerger.class);
-
-    private RegionMergeTransactionImpl rmt = null; // FIXME: Use of private type
-    private HRegion mergedRegion = null; // FIXME: Use of private type
-
-    @Override
-    public void preMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-            Region regionA, Region regionB, List<Mutation> metaEntries) throws IOException {
-        HTableDescriptor tableDesc = regionA.getTableDesc();
-        if (SchemaUtil.isSystemTable(tableDesc.getName())) {
-            return;
-        }
-        RegionServerServices rss = ctx.getEnvironment().getRegionServerServices();
-        HRegionServer rs = (HRegionServer) rss;
-        if (tableDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) == null
-                || !Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(tableDesc
-                        .getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
-            TableName indexTable =
-                    TableName.valueOf(MetaDataUtil.getLocalIndexPhysicalName(tableDesc.getName()));
-            if (!MetaTableAccessor.tableExists(rs.getConnection(), indexTable)) return;
-            Region indexRegionA = IndexUtil.getIndexRegion(regionA, ctx.getEnvironment());
-            if (indexRegionA == null) {
-                LOG.warn("Index region corresponindg to data region " + regionA
-                        + " not in the same server. So skipping the merge.");
-                ctx.bypass();
-                return;
-            }
-            Region indexRegionB = IndexUtil.getIndexRegion(regionB, ctx.getEnvironment());
-            if (indexRegionB == null) {
-                LOG.warn("Index region corresponindg to region " + regionB
-                        + " not in the same server. So skipping the merge.");
-                ctx.bypass();
-                return;
-            }
-            try {
-                rmt = new RegionMergeTransactionImpl(indexRegionA, indexRegionB, false);
-                if (!rmt.prepare(rss)) {
-                    LOG.error("Prepare for the index regions merge [" + indexRegionA + ","
-                            + indexRegionB + "] failed. So returning null. ");
-                    ctx.bypass();
-                    return;
-                }
-                this.mergedRegion = rmt.stepsBeforePONR(rss, rss, false);
-                rmt.prepareMutationsForMerge(mergedRegion.getRegionInfo(),
-                    indexRegionA.getRegionInfo(), indexRegionB.getRegionInfo(),
-                    rss.getServerName(), metaEntries,
-                    mergedRegion.getTableDesc().getRegionReplication());
-            } catch (Exception e) {
-                ctx.bypass();
-                LOG.warn("index regions merge failed with the exception ", e);
-                if (rmt != null) {
-                    rmt.rollback(rss, rss);
-                    rmt = null;
-                    mergedRegion = null;
-                }
-            }
-        }
-    }
-
-    @Override
-    public void postMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-            Region regionA, Region regionB, Region mergedRegion) throws IOException {
-        if (rmt != null && this.mergedRegion != null) {
-            RegionServerCoprocessorEnvironment environment = ctx.getEnvironment();
-            HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
-            rmt.stepsAfterPONR(rs, rs, this.mergedRegion);
-        }
-    }
-
-    @Override
-    public void preRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-            Region regionA, Region regionB) throws IOException {
-        HRegionServer rs = (HRegionServer) ctx.getEnvironment().getRegionServerServices();
-        try {
-            if (rmt != null) {
-                rmt.rollback(rs, rs);
-                rmt = null;
-                mergedRegion = null;
-            }
-        } catch (Exception e) {
-            LOG.error("Error while rolling back the merge failure for index regions", e);
-            rs.abort("Abort; we got an error during rollback of index");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexSplitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexSplitter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexSplitter.java
index ba158a8..c60058c 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexSplitter.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexSplitter.java
@@ -17,145 +17,8 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.PairOfSameType;
-import org.apache.phoenix.hbase.index.util.VersionUtil;
-import org.apache.phoenix.parse.ParseNodeFactory;
-import org.apache.phoenix.schema.types.PBoolean;
-import org.apache.phoenix.util.IndexUtil;
-import org.apache.phoenix.util.MetaDataUtil;
-import org.apache.phoenix.util.SchemaUtil;
-
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.List;
 
 public class LocalIndexSplitter extends BaseRegionObserver {
-
-    private static final Log LOG = LogFactory.getLog(LocalIndexSplitter.class);
-
-    private SplitTransactionImpl st = null; // FIXME: Uses private type
-    private PairOfSameType<Region> daughterRegions = null;
-    private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
-    private static final int SPLIT_TXN_MINIMUM_SUPPORTED_VERSION = VersionUtil
-            .encodeVersion("0.98.9");
-
-    @Override
-    public void preSplitBeforePONR(ObserverContext<RegionCoprocessorEnvironment> ctx,
-            byte[] splitKey, List<Mutation> metaEntries) throws IOException {
-        RegionCoprocessorEnvironment environment = ctx.getEnvironment();
-        HTableDescriptor tableDesc = ctx.getEnvironment().getRegion().getTableDesc();
-        if (SchemaUtil.isSystemTable(tableDesc.getName())) {
-            return;
-        }
-        final RegionServerServices rss = ctx.getEnvironment().getRegionServerServices();
-        if (tableDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) == null
-                || !Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(tableDesc
-                        .getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
-            TableName indexTable =
-                    TableName.valueOf(MetaDataUtil.getLocalIndexPhysicalName(tableDesc.getName()));
-            if (!MetaTableAccessor.tableExists(rss.getConnection(), indexTable)) return;
-
-            Region indexRegion = IndexUtil.getIndexRegion(environment);
-            if (indexRegion == null) {
-                LOG.warn("Index region corresponindg to data region " + environment.getRegion()
-                        + " not in the same server. So skipping the split.");
-                ctx.bypass();
-                return;
-            }
-            // FIXME: Uses private type
-            try {
-                int encodedVersion = VersionUtil.encodeVersion(environment.getHBaseVersion());
-                if(encodedVersion >= SPLIT_TXN_MINIMUM_SUPPORTED_VERSION) {
-                    st = new SplitTransactionImpl(indexRegion, splitKey);
-                    st.useZKForAssignment =
-                            environment.getConfiguration().getBoolean("hbase.assignment.usezk",
-                                true);
-                } else {
-                    st = new IndexSplitTransaction(indexRegion, splitKey);
-                }
-
-                if (!st.prepare()) {
-                    LOG.error("Prepare for the table " + indexRegion.getTableDesc().getNameAsString()
-                        + " failed. So returning null. ");
-                    ctx.bypass();
-                    return;
-                }
-                ((HRegion)indexRegion).forceSplit(splitKey);
-                User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
-                  @Override
-                  public Void run() throws Exception {                  
-                    daughterRegions = st.stepsBeforePONR(rss, rss, false);
-                    return null;
-                  }
-                });
-                HRegionInfo copyOfParent = new HRegionInfo(indexRegion.getRegionInfo());
-                copyOfParent.setOffline(true);
-                copyOfParent.setSplit(true);
-                // Put for parent
-                Put putParent = MetaTableAccessor.makePutFromRegionInfo(copyOfParent);
-                MetaTableAccessor.addDaughtersToPut(putParent,
-                        daughterRegions.getFirst().getRegionInfo(),
-                        daughterRegions.getSecond().getRegionInfo());
-                metaEntries.add(putParent);
-                // Puts for daughters
-                Put putA = MetaTableAccessor.makePutFromRegionInfo(
-                        daughterRegions.getFirst().getRegionInfo());
-                Put putB = MetaTableAccessor.makePutFromRegionInfo(
-                        daughterRegions.getSecond().getRegionInfo());
-                st.addLocation(putA, rss.getServerName(), 1);
-                st.addLocation(putB, rss.getServerName(), 1);
-                metaEntries.add(putA);
-                metaEntries.add(putB);
-            } catch (Exception e) {
-                ctx.bypass();
-                LOG.warn("index region splitting failed with the exception ", e);
-                if (st != null){
-                    st.rollback(rss, rss);
-                    st = null;
-                    daughterRegions = null;
-                }
-            }
-        }
-    }
-
-    @Override
-    public void preSplitAfterPONR(ObserverContext<RegionCoprocessorEnvironment> ctx)
-            throws IOException {
-        if (st == null || daughterRegions == null) return;
-        RegionCoprocessorEnvironment environment = ctx.getEnvironment();
-        HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
-        st.stepsAfterPONR(rs, rs, daughterRegions);
-    }
     
-    @Override
-    public void preRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx)
-            throws IOException {
-        RegionCoprocessorEnvironment environment = ctx.getEnvironment();
-        HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
-        try {
-            if (st != null) {
-                st.rollback(rs, rs);
-                st = null;
-                daughterRegions = null;
-            }
-        } catch (Exception e) {
-            if (st != null) {
-                LOG.error("Error while rolling back the split failure for index region", e);
-            }
-            rs.abort("Abort; we got an error during rollback of index");
-        }
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index e032feb..b545156 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -42,6 +42,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 import org.apache.phoenix.parse.BindParseNode;
+import org.apache.phoenix.parse.ColumnDef;
 import org.apache.phoenix.parse.ColumnParseNode;
 import org.apache.phoenix.parse.CreateTableStatement;
 import org.apache.phoenix.parse.ParseNode;
@@ -49,6 +50,7 @@ import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.DelegateConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PDatum;
@@ -90,6 +92,15 @@ public class CreateTableCompiler {
         String viewStatementToBe = null;
         byte[][] viewColumnConstantsToBe = null;
         BitSet isViewColumnReferencedToBe = null;
+        // Check whether column families having local index column family suffix or not if present
+        // don't allow creating table.
+        for(ColumnDef columnDef: create.getColumnDefs()) {
+            if(columnDef.getColumnDefName().getFamilyName()!=null && columnDef.getColumnDefName().getFamilyName().contains(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNALLOWED_COLUMN_FAMILY)
+                .build().buildException();
+            }
+        }
+
         if (type == PTableType.VIEW) {
             TableRef tableRef = resolver.getTables().get(0);
             int nColumns = tableRef.getTable().getColumns().size();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
index 8d7d7cf..99a9731 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
@@ -287,14 +287,22 @@ public class ProjectionCompiler {
             String indexColName = IndexUtil.getIndexColumnName(column);
             PColumn indexColumn = null;
             ColumnRef ref = null;
+            String indexColumnFamily = null;
             try {
                 indexColumn = index.getColumn(indexColName);
                 ref = new ColumnRef(tableRef, indexColumn.getPosition());
+                indexColumnFamily = indexColumn.getFamilyName() == null ? null : indexColumn.getFamilyName().getString();
             } catch (ColumnNotFoundException e) {
                 if (index.getIndexType() == IndexType.LOCAL) {
                     try {
                         ref = new LocalIndexDataColumnRef(context, indexColName);
                         indexColumn = ref.getColumn();
+                        indexColumnFamily =
+                                indexColumn.getFamilyName() == null ? null
+                                        : (index.getIndexType() == IndexType.LOCAL ? IndexUtil
+                                                .getLocalIndexColumnFamily(indexColumn
+                                                        .getFamilyName().getString()) : indexColumn
+                                                .getFamilyName().getString());
                     } catch (ColumnFamilyNotFoundException c) {
                         throw e;
                     }
@@ -303,7 +311,7 @@ public class ProjectionCompiler {
                 }
             }
             if (resolveColumn) {
-                ref = context.getResolver().resolveColumn(index.getTableName().getString(), indexColumn.getFamilyName() == null ? null : indexColumn.getFamilyName().getString(), indexColName);
+                ref = context.getResolver().resolveColumn(index.getTableName().getString(), indexColumnFamily, indexColName);
             }
             Expression expression = ref.newColumnExpression();
             projectedExpressions.add(expression);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index e2fc2ca..7d60cd5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -32,6 +32,11 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.client.HRegionLocator;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -50,6 +55,7 @@ import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.index.IndexMetaDataCacheClient;
 import org.apache.phoenix.index.PhoenixIndexCodec;
 import org.apache.phoenix.iterate.ResultIterator;
@@ -81,11 +87,14 @@ import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ReadOnlyTableException;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.tuple.Tuple;
@@ -104,7 +113,7 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 public class UpsertCompiler {
-    private static void setValues(byte[][] values, int[] pkSlotIndex, int[] columnIndexes, PTable table, Map<ImmutableBytesPtr,RowMutationState> mutation, PhoenixStatement statement, boolean useServerTimestamp) {
+    private static void setValues(byte[][] values, int[] pkSlotIndex, int[] columnIndexes, PTable table, Map<ImmutableBytesPtr,RowMutationState> mutation, PhoenixStatement statement, boolean useServerTimestamp) throws SQLException {
         Map<PColumn,byte[]> columnValues = Maps.newHashMapWithExpectedSize(columnIndexes.length);
         byte[][] pkValues = new byte[table.getPKColumns().size()][];
         // If the table uses salting, the first byte is the salting byte, set to an empty array

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
----------------------------------------------------------------------
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 fa4343a..39ac6fe 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
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
index 3b8efc3..2d7c291 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
@@ -63,7 +63,6 @@ import org.apache.phoenix.join.HashJoinInfo;
 import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.StaleRegionBoundaryCacheException;
 import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.Closeables;
@@ -129,7 +128,6 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         byte[] localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD);
         List<IndexMaintainer> indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes);
         TupleProjector tupleProjector = null;
-        Region dataRegion = null;
         byte[][] viewConstants = null;
         ColumnReference[] dataColumns = IndexUtil.deserializeDataTableColumnsToJoin(scan);
 
@@ -138,13 +136,12 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         if (ScanUtil.isLocalIndex(scan) || (j == null && p != null)) {
             if (dataColumns != null) {
                 tupleProjector = IndexUtil.getTupleProjector(scan, dataColumns);
-                dataRegion = IndexUtil.getDataRegion(c.getEnvironment());
                 viewConstants = IndexUtil.deserializeViewConstantsFromScan(scan);
             }
             ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
             innerScanner =
                     getWrappedScanner(c, innerScanner, offset, scan, dataColumns, tupleProjector,
-                            dataRegion, indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr);
+                        c.getEnvironment().getRegion(), indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr);
         }
 
         if (j != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 8264101..e77ff8b 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
@@ -1608,7 +1608,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     }
                     results.add(result);
                 }
-                TableViewFinderResult tableViewFinderResult = new TableViewFinderResult(results);
+                TableViewFinderResult tableViewFinderResult = new TableViewFinderResult(results, table);
                 if (numOfChildViews > 0 && !allViewsInCurrentRegion) {
                     tableViewFinderResult.setAllViewsNotInSingleRegion();
                 }
@@ -3377,13 +3377,21 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
         private List<Result> results = Lists.newArrayList();
         private boolean allViewsNotInSingleRegion = false;
+        private PTable table;
 
-        private TableViewFinderResult(List<Result> results) {
+        private TableViewFinderResult(List<Result> results, PTable table) {
             this.results = results;
+            this.table = table;
         }
 
         public boolean hasViews() {
-            return results.size() > 0;
+            int localIndexesCount = 0;
+            for(PTable index : table.getIndexes()) {
+                if(index.getIndexType().equals(IndexType.LOCAL)) {
+                    localIndexesCount++;
+                }
+            }
+            return results.size()-localIndexesCount > 0;
         }
 
         private void setAllViewsNotInSingleRegion() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
index 35be54d..48e3704 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
@@ -209,7 +209,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
         ColumnReference[] dataColumns = IndexUtil.deserializeDataTableColumnsToJoin(scan);
         if (dataColumns != null) {
             tupleProjector = IndexUtil.getTupleProjector(scan, dataColumns);
-            dataRegion = IndexUtil.getDataRegion(c.getEnvironment());
+            dataRegion = c.getEnvironment().getRegion();
             byte[] localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD);
             List<IndexMaintainer> indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes);
             indexMaintainer = indexMaintainers.get(0);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
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 f817772..d474665 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
@@ -165,6 +165,9 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             throws IOException {
         s = super.preScannerOpen(e, scan, s);
         if (ScanUtil.isAnalyzeTable(scan)) {
+            if (!ScanUtil.isLocalIndex(scan)) {
+                scan.getFamilyMap().clear();
+            }
             // We are setting the start row and stop row such that it covers the entire region. As part
             // of Phonenix-1263 we are storing the guideposts against the physical table rather than
             // individual tenant specific tables.
@@ -180,6 +183,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         RegionCoprocessorEnvironment env = c.getEnvironment();
         Region region = env.getRegion();
         long ts = scan.getTimeRange().getMax();
+        boolean localIndexScan = ScanUtil.isLocalIndex(scan);
         if (ScanUtil.isAnalyzeTable(scan)) {
             byte[] gp_width_bytes =
                     scan.getAttribute(BaseScannerRegionObserver.GUIDEPOST_WIDTH_BYTES);
@@ -192,7 +196,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             return collectStats(s, statsCollector, region, scan, env.getConfiguration());
         }
         int offsetToBe = 0;
-        if (ScanUtil.isLocalIndex(scan)) {
+        if (localIndexScan) {
             /*
              * For local indexes, we need to set an offset on row key expressions to skip
              * the region start key.
@@ -202,7 +206,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             ScanUtil.setRowKeyOffset(scan, offsetToBe);
         }
         final int offset = offsetToBe;
-
+        
         PTable projectedTable = null;
         PTable writeToTable = null;
         byte[][] values = null;
@@ -238,6 +242,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             projectedTable = deserializeTable(upsertSelectTable);
             selectExpressions = deserializeExpressions(scan.getAttribute(BaseScannerRegionObserver.UPSERT_SELECT_EXPRS));
             values = new byte[projectedTable.getPKColumns().size()][];
+            
         } else {
             byte[] isDeleteAgg = scan.getAttribute(BaseScannerRegionObserver.DELETE_AGG);
             isDelete = isDeleteAgg != null && Bytes.compareTo(PDataType.TRUE_BYTES, isDeleteAgg) == 0;
@@ -248,22 +253,19 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             emptyCF = scan.getAttribute(BaseScannerRegionObserver.EMPTY_CF);
         }
         TupleProjector tupleProjector = null;
-        Region dataRegion = null;
         byte[][] viewConstants = null;
         ColumnReference[] dataColumns = IndexUtil.deserializeDataTableColumnsToJoin(scan);
-        boolean localIndexScan = ScanUtil.isLocalIndex(scan);
         final TupleProjector p = TupleProjector.deserializeProjectorFromScan(scan);
         final HashJoinInfo j = HashJoinInfo.deserializeHashJoinFromScan(scan);
         if ((localIndexScan && !isDelete && !isDescRowKeyOrderUpgrade) || (j == null && p != null)) {
             if (dataColumns != null) {
                 tupleProjector = IndexUtil.getTupleProjector(scan, dataColumns);
-                dataRegion = IndexUtil.getDataRegion(env);
                 viewConstants = IndexUtil.deserializeViewConstantsFromScan(scan);
             }
             ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
             theScanner =
                     getWrappedScanner(c, theScanner, offset, scan, dataColumns, tupleProjector, 
-                            dataRegion, indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr);
+                        region, indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr);
         } 
         
         if (j != null)  {
@@ -513,7 +515,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                             // Commit in batches based on UPSERT_BATCH_SIZE_ATTRIB in config
                             if (!indexMutations.isEmpty() && batchSize > 0 &&
                                     indexMutations.size() % batchSize == 0) {
-                                commitIndexMutations(c, region, indexMutations);
+                                commitBatch(region, indexMutations, null);
+                                indexMutations.clear();
                             }
                         } catch (ConstraintViolationException e) {
                             // Log and ignore in count
@@ -544,7 +547,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         }
 
         if (!indexMutations.isEmpty()) {
-            commitIndexMutations(c, region, indexMutations);
+            commitBatch(region,indexMutations, null);
+            indexMutations.clear();
         }
 
         final boolean hadAny = hasAny;
@@ -579,31 +583,6 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         return scanner;
     }
 
-    private void commitIndexMutations(final ObserverContext<RegionCoprocessorEnvironment> c,
-            Region region, List<Mutation> indexMutations) throws IOException {
-        // Get indexRegion corresponding to data region
-        Region indexRegion = IndexUtil.getIndexRegion(c.getEnvironment());
-        if (indexRegion != null) {
-            commitBatch(indexRegion, indexMutations, null);
-        } else {
-            TableName indexTable =
-                    TableName.valueOf(MetaDataUtil.getLocalIndexPhysicalName(region.getTableDesc()
-                            .getName()));
-            HTableInterface table = null;
-            try {
-                table = c.getEnvironment().getTable(indexTable);
-                table.batch(indexMutations);
-            } catch (InterruptedException ie) {
-                Thread.currentThread().interrupt();
-                ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionInfo().getRegionNameAsString(),
-                    ie);
-            } finally {
-                if (table != null) table.close();
-             }
-        }
-        indexMutations.clear();
-    }
-
     @Override
     public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, final Store store,
             InternalScanner scanner, final ScanType scanType) throws IOException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index 43965f5..5a8fffa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -23,6 +23,7 @@ import java.util.Map;
 
 import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.AmbiguousColumnException;
 import org.apache.phoenix.schema.AmbiguousTableException;
@@ -203,6 +204,7 @@ public enum SQLExceptionCode {
     NULLABLE_FIXED_WIDTH_LAST_PK(1023, "42J04", "Cannot add column to table when the last PK column is nullable and fixed width."),
     CANNOT_MODIFY_VIEW_PK(1036, "42J04", "Cannot modify the primary key of a VIEW if last PK column of parent is variable length."),
     BASE_TABLE_COLUMN(1037, "42J04", "Cannot modify columns of base table used by tenant-specific tables."),
+    UNALLOWED_COLUMN_FAMILY(1090, "42J04", "Column family names should not contain local index column prefix: "+QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX),
     // Key/value column related errors
     KEY_VALUE_NOT_NULL(1007, "42K01", "A key/value column may not be declared as not null."),
     // View related errors.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionSplitPolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionSplitPolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionSplitPolicy.java
index 8604784..62c13a5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionSplitPolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionSplitPolicy.java
@@ -17,20 +17,19 @@
  */
 package org.apache.phoenix.hbase.index;
 
-import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
+import org.apache.hadoop.hbase.regionserver.IncreasingToUpperBoundRegionSplitPolicy;
+import org.apache.phoenix.query.QueryConstants;
 
 /**
  * Split policy for index regions to avoid split from external requests.
  */
-public class IndexRegionSplitPolicy extends RegionSplitPolicy {
+public class IndexRegionSplitPolicy extends IncreasingToUpperBoundRegionSplitPolicy {
 
     @Override
-    protected boolean shouldSplit() {
+    protected boolean skipStoreFileRangeCheck(String familyName) {
+        if (familyName.startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+            return true;
+        }
         return false;
     }
-
-    protected boolean skipStoreFileRangeCheck() {
-        return true;
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
index 18384fa..2811c43 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
@@ -23,8 +23,11 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -49,6 +52,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.hbase.index.builder.IndexBuildManager;
 import org.apache.phoenix.hbase.index.builder.IndexBuilder;
@@ -64,6 +68,7 @@ import org.apache.phoenix.hbase.index.write.recovery.StoreFailuresInCachePolicy;
 import org.apache.phoenix.hbase.index.write.recovery.TrackingParallelWriterIndexCommitter;
 import org.apache.phoenix.trace.TracingUtils;
 import org.apache.phoenix.trace.util.NullSpan;
+import org.apache.phoenix.util.IndexUtil;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
@@ -92,6 +97,7 @@ public class Indexer extends BaseRegionObserver {
 
   protected IndexWriter writer;
   protected IndexBuildManager builder;
+  private RegionCoprocessorEnvironment environment;
 
   /** Configuration key for the {@link IndexBuilder} to use */
   public static final String INDEX_BUILDER_CONF_KEY = "index.builder";
@@ -133,6 +139,7 @@ public class Indexer extends BaseRegionObserver {
   public void start(CoprocessorEnvironment e) throws IOException {
       try {
         final RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) e;
+        this.environment = env;
         String serverName = env.getRegionServerServices().getServerName().getServerName();
         if (env.getConfiguration().getBoolean(CHECK_VERSION_CONF_KEY, true)) {
           // make sure the right version <-> combinations are allowed.
@@ -297,7 +304,7 @@ public class Indexer extends BaseRegionObserver {
     // update right away
     if (durability == Durability.SKIP_WAL) {
       try {
-        this.writer.write(indexUpdates);
+        this.writer.write(indexUpdates, false);
         return false;
       } catch (Throwable e) {
         LOG.error("Failed to update index with entries:" + indexUpdates, e);
@@ -320,7 +327,7 @@ public class Indexer extends BaseRegionObserver {
       super.postPut(e, put, edit, durability);
           return;
         }
-    doPost(edit, put, durability);
+    doPost(edit, put, durability, true);
   }
 
   @Override
@@ -330,7 +337,7 @@ public class Indexer extends BaseRegionObserver {
       super.postDelete(e, delete, edit, durability);
           return;
         }
-    doPost(edit, delete, durability);
+    doPost(edit, delete, durability, true);
   }
 
   @Override
@@ -345,12 +352,12 @@ public class Indexer extends BaseRegionObserver {
     //each batch operation, only the first one will have anything useful, so we can just grab that
     Mutation mutation = miniBatchOp.getOperation(0);
     WALEdit edit = miniBatchOp.getWalEdit(0);
-    doPost(edit, mutation, mutation.getDurability());
+    doPost(edit, mutation, mutation.getDurability(), false);
   }
 
-  private void doPost(WALEdit edit, Mutation m, final Durability durability) throws IOException {
+  private void doPost(WALEdit edit, Mutation m, final Durability durability, boolean allowLocalUpdates) throws IOException {
     try {
-      doPostWithExceptions(edit, m, durability);
+      doPostWithExceptions(edit, m, durability, allowLocalUpdates);
       return;
     } catch (Throwable e) {
       rethrowIndexingException(e);
@@ -359,7 +366,7 @@ public class Indexer extends BaseRegionObserver {
         "Somehow didn't complete the index update, but didn't return succesfully either!");
   }
 
-  private void doPostWithExceptions(WALEdit edit, Mutation m, final Durability durability)
+  private void doPostWithExceptions(WALEdit edit, Mutation m, final Durability durability, boolean allowLocalUpdates)
           throws Exception {
       //short circuit, if we don't need to do any work
       if (durability == Durability.SKIP_WAL || !this.builder.isEnabled(m)) {
@@ -393,17 +400,31 @@ public class Indexer extends BaseRegionObserver {
            * once (this hook gets called with the same WALEdit for each Put/Delete in a batch, which can
            * lead to writing all the index updates for each Put/Delete).
            */
-          if (!ikv.getBatchFinished()) {
+          if (!ikv.getBatchFinished() || allowLocalUpdates) {
               Collection<Pair<Mutation, byte[]>> indexUpdates = extractIndexUpdate(edit);
 
               // the WAL edit is kept in memory and we already specified the factory when we created the
               // references originally - therefore, we just pass in a null factory here and use the ones
               // already specified on each reference
               try {
-                  current.addTimelineAnnotation("Actually doing index update for first time");
-                  writer.writeAndKillYourselfOnFailure(indexUpdates);
-              } finally {
-                  // With a custom kill policy, we may throw instead of kill the server.
+            	  if (!ikv.getBatchFinished()) {
+            		  current.addTimelineAnnotation("Actually doing index update for first time");
+            		  writer.writeAndKillYourselfOnFailure(indexUpdates, allowLocalUpdates);
+            	  } else if (allowLocalUpdates) {
+            		  Collection<Pair<Mutation, byte[]>> localUpdates =
+            				  new ArrayList<Pair<Mutation, byte[]>>();
+            		  current.addTimelineAnnotation("Actually doing local index update for first time");
+            		  for (Pair<Mutation, byte[]> mutation : indexUpdates) {
+            			  if (Bytes.toString(mutation.getSecond()).equals(
+            					  environment.getRegion().getTableDesc().getNameAsString())) {
+            				  localUpdates.add(mutation);
+            			  }
+            		  }
+                      if(!localUpdates.isEmpty()) {
+                    	  writer.writeAndKillYourselfOnFailure(localUpdates, allowLocalUpdates);
+                      }
+            	  }
+              } finally {                  // With a custom kill policy, we may throw instead of kill the server.
                   // Without doing this in a finally block (at least with the mini cluster),
                   // the region server never goes down.
 
@@ -465,7 +486,7 @@ public class Indexer extends BaseRegionObserver {
     // do the usual writer stuff, killing the server again, if we can't manage to make the index
     // writes succeed again
     try {
-        writer.writeAndKillYourselfOnFailure(updates);
+        writer.writeAndKillYourselfOnFailure(updates, true);
     } catch (IOException e) {
         LOG.error("Exception thrown instead of killing server during index writing", e);
     }
@@ -489,7 +510,7 @@ public class Indexer extends BaseRegionObserver {
      * hopes they come up before the primary table finishes.
      */
     Collection<Pair<Mutation, byte[]>> indexUpdates = extractIndexUpdate(logEdit);
-    recoveryWriter.write(indexUpdates);
+    recoveryWriter.write(indexUpdates, true);
   }
 
   /**


[4/5] phoenix git commit: PHOENIX-1734 Local index improvements(Rajeshbabu)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
----------------------------------------------------------------------
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 f7edea7..416c452 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
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
@@ -35,9 +34,8 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -47,12 +45,6 @@ import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
-import org.apache.hadoop.hbase.regionserver.LocalIndexSplitter;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.QueryPlan;
@@ -63,19 +55,16 @@ import org.apache.phoenix.hbase.index.IndexRegionSplitPolicy;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTableKey;
-import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.ByteUtil;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
-import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -88,9 +77,6 @@ import com.google.common.collect.Maps;
 @RunWith(Parameterized.class)
 public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
-    private static CountDownLatch latch1 = new CountDownLatch(1);
-    private static CountDownLatch latch2 = new CountDownLatch(1);
-    private static final int WAIT_TIME_SECONDS = 60;
     private String schemaName="TEST";
     private boolean isNamespaceMapped;
     private String tableName = schemaName + ".T";
@@ -102,8 +88,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
     public LocalIndexIT(boolean isNamespaceMapped) {
         this.isNamespaceMapped = isNamespaceMapped;
         this.physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
-        this.indexPhysicalTableName = Bytes
-                .toString(MetaDataUtil.getLocalIndexPhysicalName(physicalTableName.getName()));
+        this.indexPhysicalTableName = this.physicalTableName.getNameAsString();
     }
     
     @BeforeClass 
@@ -150,6 +135,15 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
     }
 
     @Test
+    public void testCreationOfTableWithLocalIndexColumnFamilyPrefixShouldFail() throws Exception {
+        Connection conn1 = DriverManager.getConnection(getUrl());
+        try {
+            conn1.createStatement().execute("CREATE TABLE T(L#a varchar primary key, aL# integer)");
+            fail("Column families specified in the table creation should not have local colunm prefix.");
+        } catch (SQLException e) { }
+    }
+
+    @Test
     public void testLocalIndexCreationWithSplitsShouldFail() throws Exception {
         createBaseTable(tableName, null, null);
         Connection conn1 = getConnection();
@@ -215,11 +209,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
         Connection conn2 = getConnection();
         conn1.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + tableName + "(v1)");
         conn2.createStatement().executeQuery("SELECT * FROM " + tableName).next();
-        HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
-        assertTrue("Local index table should be present.", admin.tableExists(indexPhysicalTableName));
         conn1.createStatement().execute("DROP TABLE "+ tableName);
-        admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
-        assertFalse("Local index table should be deleted.", admin.tableExists(indexPhysicalTableName));
         ResultSet rs = conn2.createStatement().executeQuery("SELECT "
                 + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + ","
                 + PhoenixDatabaseMetaData.SEQUENCE_NAME
@@ -247,6 +237,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
         byte[][] endKeys = startEndKeys.getSecond();
         for (int i = 0; i < startKeys.length; i++) {
             Scan s = new Scan();
+            s.addFamily(QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES);
             s.setStartRow(startKeys[i]);
             s.setStopRow(endKeys[i]);
             ResultScanner scanner = indexTable.getScanner(s);
@@ -280,6 +271,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
         byte[][] endKeys = startEndKeys.getSecond();
         for (int i = 0; i < startKeys.length; i++) {
             Scan s = new Scan();
+            s.addFamily(QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES);
             s.setStartRow(startKeys[i]);
             s.setStopRow(endKeys[i]);
             ResultScanner scanner = indexTable.getScanner(s);
@@ -318,7 +310,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             
             assertEquals(
                 "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER "
-                        + indexPhysicalTableName + " [-32768,'a'] - [-32768,'b']\n"
+                        + indexPhysicalTableName + " [1,'a'] - [1,'b']\n"
                                 + "    SERVER FILTER BY FIRST KEY ONLY\n"
                                 + "CLIENT MERGE SORT",
                         QueryUtil.getExplainPlan(rs));
@@ -342,7 +334,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             
             assertEquals(
                 "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER "
-                        + indexPhysicalTableName + " [-32768,'a']\n"
+                        + indexPhysicalTableName + " [1,'a']\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY\n"
                         + "CLIENT MERGE SORT",
                         QueryUtil.getExplainPlan(rs));
@@ -361,7 +353,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             rs = conn1.createStatement().executeQuery("EXPLAIN "+ query);
             
             assertEquals("CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER " + indexPhysicalTableName
-                    + " [-32768,*] - [-32768,'z']\n" + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                    + " [1,*] - [1,'z']\n" + "    SERVER FILTER BY FIRST KEY ONLY\n"
                     + "    SERVER SORTED BY [\"K3\"]\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
  
             rs = conn1.createStatement().executeQuery(query);
@@ -384,7 +376,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             
             assertEquals(
                 "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER "
-                        + indexPhysicalTableName +" [-32768]\n"
+                        + indexPhysicalTableName +" [1]\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY\n"
                         + "CLIENT MERGE SORT",
                 QueryUtil.getExplainPlan(rs));
@@ -448,7 +440,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             
             assertEquals(
                 "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER "
-                        + indexPhysicalTableName + " [-32768,'a']\n"
+                        + indexPhysicalTableName + " [1,'a']\n"
                                 + "    SERVER FILTER BY FIRST KEY ONLY\n"
                                 + "CLIENT MERGE SORT",
                         QueryUtil.getExplainPlan(rs));
@@ -471,7 +463,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             
             assertEquals(
                 "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER "
-                        + indexPhysicalTableName +" [-32768,*] - [-32768,'z']\n"
+                        + indexPhysicalTableName +" [1,*] - [1,'z']\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY\n"
                          + "CLIENT MERGE SORT",
                 QueryUtil.getExplainPlan(rs));
@@ -507,7 +499,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             
             assertEquals(
                 "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER "
-                        + indexPhysicalTableName +" [-32768,*] - [-32768,'z']\n"
+                        + indexPhysicalTableName +" [1,*] - [1,'z']\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY\n"
                         + "    SERVER AGGREGATE INTO DISTINCT ROWS BY [\"V1\", \"T_ID\", \"K3\"]\nCLIENT MERGE SORT",
                 QueryUtil.getExplainPlan(rs));
@@ -535,7 +527,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             rs = conn1.createStatement().executeQuery("EXPLAIN " + query);
             assertEquals(
                 "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER "
-                        + indexPhysicalTableName +" [-32768,*] - [-32768,'z']\n"
+                        + indexPhysicalTableName +" [1,*] - [1,'z']\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY\n"
                         + "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [\"V1\"]\nCLIENT MERGE SORT",
                 QueryUtil.getExplainPlan(rs));
@@ -591,8 +583,8 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             conn1.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + tableName + "(v1)");
             conn1.createStatement().execute("DROP INDEX " + indexName + " ON " + tableName);
             HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
-            HTable indexTable = new HTable(admin.getConfiguration() ,TableName.valueOf(MetaDataUtil.getLocalIndexTableName(tableName)));
-            Pair<byte[][], byte[][]> startEndKeys = indexTable.getStartEndKeys();
+            HTable table = new HTable(admin.getConfiguration() ,TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME));
+            Pair<byte[][], byte[][]> startEndKeys = table.getStartEndKeys();
             byte[][] startKeys = startEndKeys.getFirst();
             byte[][] endKeys = startEndKeys.getSecond();
             // No entry should be present in local index table after drop index.
@@ -600,7 +592,13 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
                 Scan s = new Scan();
                 s.setStartRow(startKeys[i]);
                 s.setStopRow(endKeys[i]);
-                ResultScanner scanner = indexTable.getScanner(s);
+                Collection<HColumnDescriptor> families = table.getTableDescriptor().getFamilies();
+                for(HColumnDescriptor cf: families) {
+                    if(cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)){
+                        s.addFamily(cf.getName());
+                    }
+                }
+                ResultScanner scanner = table.getScanner(s);
                 int count = 0;
                 for(Result r:scanner){
                     count++;
@@ -608,7 +606,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
                 scanner.close();
                 assertEquals(0, count);
             }
-            indexTable.close();
+            table.close();
         } finally {
             conn1.close();
         }
@@ -720,7 +718,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             rs = conn1.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("j", rs.getString("t_id"));
-            assertTrue(rs.next());
+            assertTrue(rs.next());     
             assertEquals("b", rs.getString("t_id"));
             assertFalse(rs.next());
        } finally {
@@ -778,18 +776,6 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
                             admin.getConnection(), physicalTableName, false);
                 }
                 assertEquals(4+i, regionsOfUserTable.size());
-                TableName indexTable =
-                        TableName.valueOf(indexPhysicalTableName);
-                List<HRegionInfo> regionsOfIndexTable =
-                        MetaTableAccessor.getTableRegions(getUtility().getZooKeeperWatcher(),
-                                admin.getConnection(), indexTable, false);
-
-                while (regionsOfIndexTable.size() != (4 + i)) {
-                    Thread.sleep(100);
-                    regionsOfIndexTable = MetaTableAccessor.getTableRegions(getUtility().getZooKeeperWatcher(),
-                            admin.getConnection(), indexTable, false);
-                }
-                assertEquals(4 + i, regionsOfIndexTable.size());
                 String[] tIdColumnValues = new String[26]; 
                 String[] v1ColumnValues = new String[26];
                 int[] k1ColumnValue = new int[26];
@@ -814,7 +800,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
                 rs = conn1.createStatement().executeQuery("EXPLAIN " + query);
                 assertEquals(
                         "CLIENT PARALLEL " + (4 + i) + "-WAY RANGE SCAN OVER "
-                                + indexPhysicalTableName + " [-32768]\n"
+                                + indexPhysicalTableName + " [1]\n"
                                         + "    SERVER FILTER BY FIRST KEY ONLY\n"
                                 + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
                 
@@ -824,7 +810,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
                     "CLIENT PARALLEL "
                             + ((strings[3 * i].compareTo("j") < 0) ? (4 + i) : (4 + i - 1))
                             + "-WAY RANGE SCAN OVER "
-                            + indexPhysicalTableName + " [-32767]\n"
+                            + indexPhysicalTableName + " [2]\n"
                                     + "    SERVER FILTER BY FIRST KEY ONLY\n"
                             + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
                 rs = conn1.createStatement().executeQuery(query);
@@ -927,19 +913,6 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
                 regionsOfUserTable = MetaTableAccessor.getTableRegions(getUtility().getZooKeeperWatcher(),
                         admin.getConnection(), physicalTableName, false);
             }
-            assertEquals(3, regionsOfUserTable.size());
-            TableName indexTable =
-                    TableName.valueOf(indexPhysicalTableName);
-            List<HRegionInfo> regionsOfIndexTable =
-                    MetaTableAccessor.getTableRegions(getUtility().getZooKeeperWatcher(),
-                            admin.getConnection(), indexTable, false);
-
-            while (regionsOfIndexTable.size() != 3) {
-                Thread.sleep(100);
-                regionsOfIndexTable = MetaTableAccessor.getTableRegions(
-                        getUtility().getZooKeeperWatcher(), admin.getConnection(), indexTable, false);
-            }
-            assertEquals(3, regionsOfIndexTable.size());
             String query = "SELECT t_id,k1,v1 FROM " + tableName;
             rs = conn1.createStatement().executeQuery(query);
             Thread.sleep(1000);
@@ -953,7 +926,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             assertEquals(
                 "CLIENT PARALLEL " + 3 + "-WAY RANGE SCAN OVER "
                         + indexPhysicalTableName
-                        + " [-32768]\n" + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                        + " [1]\n" + "    SERVER FILTER BY FIRST KEY ONLY\n"
                         + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
 
             query = "SELECT t_id,k1,k3 FROM " + tableName;
@@ -961,7 +934,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             assertEquals(
                 "CLIENT PARALLEL " + 3 + "-WAY RANGE SCAN OVER "
                         + indexPhysicalTableName
-                        + " [-32767]\n" + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                        + " [2]\n" + "    SERVER FILTER BY FIRST KEY ONLY\n"
                         + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
 
             rs = conn1.createStatement().executeQuery(query);
@@ -976,93 +949,4 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             conn1.close();
         }
     }
-
-    @Test
-    public void testLocalIndexStateWhenSplittingInProgress() throws Exception {
-        if (isNamespaceMapped) { return; }
-        createBaseTable(tableName+"2", null, "('e','j','o')");
-        Connection conn1 = getConnection();
-        try{
-            String[] strings = {"a","b","c","d","e","f","g","h","i","j","k","l","m","n","o","p","q","r","s","t","u","v","w","x","y","z"};
-            for (int i = 0; i < 26; i++) {
-                conn1.createStatement().execute(
-                    "UPSERT INTO " + tableName+"2" + " values('"+strings[i]+"'," + i + ","
-                            + (i + 1) + "," + (i + 2) + ",'" + strings[25 - i] + "')");
-            }
-            conn1.commit();
-            conn1.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + tableName+"2" + "(v1)");
-            conn1.createStatement().execute("CREATE LOCAL INDEX " + indexName + "_2 ON " + tableName+"2" + "(k3)");
-
-            ResultSet rs = conn1.createStatement().executeQuery("SELECT * FROM " + tableName+"2");
-            assertTrue(rs.next());
-            HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
-            HTableDescriptor tableDesc = admin.getTableDescriptor(SchemaUtil.getPhysicalTableName((tableName+"2").getBytes(),isNamespaceMapped));
-            tableDesc.removeCoprocessor(LocalIndexSplitter.class.getName());
-            tableDesc.addCoprocessor(MockedLocalIndexSplitter.class.getName(), null,
-                1, null);
-            admin.disableTable(tableDesc.getTableName());
-            admin.modifyTable(tableDesc.getTableName(), tableDesc);
-            admin.enableTable(tableDesc.getTableName());
-            TableName indexTable = TableName.valueOf(MetaDataUtil.getLocalIndexTableName(
-                    SchemaUtil.getPhysicalTableName((tableName + "2").getBytes(), isNamespaceMapped).getNameAsString()));
-            HTableDescriptor indexTableDesc = admin.getTableDescriptor(indexTable);
-            indexTableDesc.removeCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName());
-            indexTableDesc.addCoprocessor(MockedIndexHalfStoreFileReaderGenerator.class.getName(), null,
-                1, null);
-            admin.disableTable(indexTable);
-            admin.modifyTable(indexTable, indexTableDesc);
-            admin.enableTable(indexTable);
-
-            admin.split(SchemaUtil.getPhysicalTableName(Bytes.toBytes(tableName + "2"), isNamespaceMapped),
-                    ByteUtil.concat(Bytes.toBytes(strings[3])));
-            List<HRegionInfo> regionsOfUserTable =
-                    admin.getTableRegions(SchemaUtil.getPhysicalTableName(Bytes.toBytes(tableName + "2"), isNamespaceMapped));
-
-            while (regionsOfUserTable.size() != 5) {
-                Thread.sleep(100);
-                regionsOfUserTable = admin.getTableRegions(SchemaUtil.getPhysicalTableName(Bytes.toBytes(tableName + "2"), isNamespaceMapped));
-            }
-            assertEquals(5, regionsOfUserTable.size());
-
-            List<HRegionInfo> regionsOfIndexTable = admin.getTableRegions(indexTable);
-
-            while (regionsOfIndexTable.size() != 5) {
-                Thread.sleep(100);
-                regionsOfIndexTable = admin.getTableRegions(indexTable);
-            }
-
-            assertEquals(5, regionsOfIndexTable.size());
-            boolean success = latch1.await(WAIT_TIME_SECONDS, TimeUnit.SECONDS);
-            assertTrue("Timed out waiting for MockedLocalIndexSplitter.preSplitAfterPONR to complete", success);
-            latch2.countDown();
-       } finally {
-            conn1.close();
-            latch1.countDown();
-            latch2.countDown();
-        }
-    }
-
-    public static class MockedIndexHalfStoreFileReaderGenerator extends IndexHalfStoreFileReaderGenerator {
-        @Override
-        public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
-                StoreFile resultFile) throws IOException {
-            try {
-                boolean success = latch2.await(WAIT_TIME_SECONDS, TimeUnit.SECONDS);
-                assertTrue("Timed out waiting for test to complete", success);
-                super.postCompact(e, store, resultFile);
-            } catch (InterruptedException e1) {
-                Thread.currentThread().interrupt();
-                throw new RuntimeException(e1);
-            }
-        }
-    }
-
-    public static class MockedLocalIndexSplitter extends LocalIndexSplitter {
-        @Override
-        public void preSplitAfterPONR(ObserverContext<RegionCoprocessorEnvironment> ctx)
-                throws IOException {
-            super.preSplitAfterPONR(ctx);
-            latch1.countDown();
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
----------------------------------------------------------------------
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 7818bcc..5d0230b 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
@@ -32,6 +32,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -44,6 +45,7 @@ import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.BaseOwnClusterHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableType;
@@ -210,6 +212,9 @@ public class MutableIndexFailureIT extends BaseOwnClusterHBaseManagedTimeIT {
                 conn.commit();
                 fail();
             } catch (SQLException e) {
+                System.out.println();
+            }  catch(Exception e) {
+                System.out.println();
             }
 
             // Verify the metadata for index is correct.
@@ -292,8 +297,8 @@ public class MutableIndexFailureIT extends BaseOwnClusterHBaseManagedTimeIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             expectedPlan = " OVER "
                     + (localIndex
-                            ? Bytes.toString(MetaDataUtil.getLocalIndexPhysicalName(SchemaUtil
-                                    .getPhysicalTableName(fullTableName.getBytes(), isNamespaceMapped).getName()))
+                            ? Bytes.toString(SchemaUtil
+                                    .getPhysicalTableName(fullTableName.getBytes(), isNamespaceMapped).getName())
                             : SchemaUtil.getPhysicalTableName(fullIndexName.getBytes(), isNamespaceMapped).getNameAsString());
             String explainPlan = QueryUtil.getExplainPlan(rs);
             assertTrue(explainPlan.contains(expectedPlan));
@@ -336,6 +341,13 @@ public class MutableIndexFailureIT extends BaseOwnClusterHBaseManagedTimeIT {
             if (c.getEnvironment().getRegionInfo().getTable().getNameAsString().contains(INDEX_NAME) && FAIL_WRITE) {
                 throw new DoNotRetryIOException();
             }
+            Mutation operation = miniBatchOp.getOperation(0);
+            Set<byte[]> keySet = operation.getFamilyMap().keySet();
+            for(byte[] family: keySet) {
+                if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX) && FAIL_WRITE) {
+                    throw new DoNotRetryIOException();
+                }
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
index 80f1250..70bfdd5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
@@ -50,7 +50,6 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.util.ByteUtil;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -117,7 +116,7 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
             String query = "SELECT char_col1, int_col1, long_col2 from " + fullTableName;
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if (localIndex) {
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName +" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName +" [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
             }
@@ -184,7 +183,7 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
             if(localIndex) {
                 query = "SELECT b.* from " + fullTableName + " where int_col1 = 4";
                 rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName +" [-32768]\n" +
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName +" [1]\n" +
                 		"    SERVER FILTER BY TO_INTEGER(\"INT_COL1\") = 4\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
                 rs = conn.createStatement().executeQuery(query);
                 assertTrue(rs.next());
@@ -247,7 +246,7 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
 	        query = "SELECT * FROM " + fullTableName;
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if(localIndex) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
 	        } else {
 	            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
 	        }
@@ -268,7 +267,7 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
 	        query = "SELECT * FROM " + fullTableName;
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if(localIndex) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName + " [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
 	        } else {
 	            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
 	        }
@@ -289,7 +288,7 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
 	        query = "SELECT * FROM " + fullTableName;
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if(localIndex) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
 	        } else {
 	            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
 	        }
@@ -354,7 +353,7 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
 	        query = "SELECT * FROM " + fullTableName;
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if (localIndex) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName+" [-32768]\n"
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\n"
 	                    + "    SERVER FILTER BY FIRST KEY ONLY\n"
 	                    + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
 	        } else {
@@ -478,7 +477,7 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
 	        query = "SELECT * FROM " + fullTableName;
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if(localIndex) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + fullTableName+" [-32768]\n"
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\n"
 	                    + "    SERVER FILTER BY FIRST KEY ONLY\n"
 	                    + "CLIENT MERGE SORT",
 	                QueryUtil.getExplainPlan(rs));
@@ -620,28 +619,38 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
         Properties props = new Properties();
         props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(2));
         props.put(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(false));
-        try(Connection conn1 = DriverManager.getConnection(getUrl(), props)){
+        Connection conn1 = DriverManager.getConnection(getUrl());
+        HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
+        dropTable(admin, conn1);
+        try{
             String[] strings = {"a","b","c","d","e","f","g","h","i","j","k","l","m","n","o","p","q","r","s","t","u","v","w","x","y","z"};
-            HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
-            dropTable(admin, conn1);
             createTableAndLoadData(conn1, strings, isReverse);
 
             ResultSet rs = conn1.createStatement().executeQuery("SELECT * FROM " + tableName);
             assertTrue(rs.next());
             splitDuringScan(conn1, strings, admin, isReverse);
             dropTable(admin, conn1);
-       } 
+       } finally {
+           dropTable(admin, conn1);
+           if(conn1 != null) conn1.close();
+           if(admin != null) admin.close();
+       }
     }
 
     private void dropTable(HBaseAdmin admin, Connection conn) throws SQLException, IOException {
-        conn.createStatement().execute("DROP TABLE IF EXISTS "+ tableName);
-        if(admin.tableExists(tableName)) {
-            admin.disableTable(TableName.valueOf(tableName));
-            admin.deleteTable(TableName.valueOf(tableName));
-        } 
-        if(admin.tableExists(localIndex? MetaDataUtil.getLocalIndexTableName(tableName): indexName)) {
-            admin.disableTable(localIndex? MetaDataUtil.getLocalIndexTableName(tableName): indexName);
-            admin.deleteTable(localIndex? MetaDataUtil.getLocalIndexTableName(tableName): indexName);
+        try {
+            conn.createStatement().execute("DROP TABLE IF EXISTS "+ tableName);
+        } finally {
+            if(admin.tableExists(tableName)) {
+                admin.disableTable(TableName.valueOf(tableName));
+                admin.deleteTable(TableName.valueOf(tableName));
+            }
+            if(!localIndex) {
+                if(admin.tableExists(indexName)) {
+                    admin.disableTable(TableName.valueOf(indexName));
+                    admin.deleteTable(TableName.valueOf(indexName));
+                }
+            }
         }
     }
 
@@ -677,7 +686,7 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
             assertEquals(4, rs.getInt(1));
 
             TableName table = TableName.valueOf(localIndex?tableName: indexName);
-            TableName indexTable = TableName.valueOf(localIndex?MetaDataUtil.getLocalIndexTableName(tableName): indexName);
+            TableName indexTable = TableName.valueOf(localIndex?tableName: indexName);
             admin.flush(indexTable);
             boolean merged = false;
             // merge regions until 1 left
@@ -697,7 +706,7 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
                   if(!merged) {
                             List<HRegionInfo> regions =
                                     admin.getTableRegions(localIndex ? table : indexTable);
-                      System.out.println("Merging: " + regions.size());
+                      Log.info("Merging: " + regions.size());
                       admin.mergeRegions(regions.get(0).getEncodedNameAsBytes(),
                           regions.get(1).getEncodedNameAsBytes(), false);
                       merged = true;
@@ -707,16 +716,17 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
                 } catch (Exception ex) {
                   Log.info(ex);
                 }
-
-                long waitStartTime = System.currentTimeMillis();
-                // wait until merge happened
-                while (System.currentTimeMillis() - waitStartTime < 10000) {
-                  List<HRegionInfo> regions = admin.getTableRegions(indexTable);
-                  System.out.println("Waiting:" + regions.size());
-                  if (regions.size() < numRegions) {
-                    break;
-                  }
-                  Threads.sleep(1000);
+                if(!localIndex) {
+                    long waitStartTime = System.currentTimeMillis();
+                    // wait until merge happened
+                    while (System.currentTimeMillis() - waitStartTime < 10000) {
+                      List<HRegionInfo> regions = admin.getTableRegions(indexTable);
+                      Log.info("Waiting:" + regions.size());
+                      if (regions.size() < numRegions) {
+                        break;
+                      }
+                      Threads.sleep(1000);
+                    }
                 }
               }
             }
@@ -725,36 +735,6 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
         }
     }
 
-    private List<HRegionInfo> mergeRegions(HBaseAdmin admin, List<HRegionInfo> regionsOfUserTable)
-            throws IOException, InterruptedException {
-        for (int i = 2; i > 0; i--) {
-            Threads.sleep(10000);
-            admin.mergeRegions(regionsOfUserTable.get(0).getEncodedNameAsBytes(),
-                regionsOfUserTable.get(1).getEncodedNameAsBytes(), false);
-            regionsOfUserTable =
-                    MetaTableAccessor.getTableRegions(getUtility().getZooKeeperWatcher(), admin.getConnection(),
-                        TableName.valueOf(localIndex? tableName:indexName), false);
-
-            while (regionsOfUserTable.size() != i) {
-                Thread.sleep(100);
-                regionsOfUserTable = MetaTableAccessor.getTableRegions(getUtility().getZooKeeperWatcher(),
-                    admin.getConnection(), TableName.valueOf(localIndex? tableName:indexName), false);
-            }
-            assertEquals(i, regionsOfUserTable.size());
-            if(localIndex) {
-                List<HRegionInfo> regionsOfIndexTable = MetaTableAccessor.getTableRegions(getUtility().getZooKeeperWatcher(),
-                    admin.getConnection(), TableName.valueOf(MetaDataUtil.getLocalIndexTableName(tableName)), false);
-               while (regionsOfIndexTable.size() != i) {
-                   Thread.sleep(100);
-                   regionsOfIndexTable = MetaTableAccessor.getTableRegions(getUtility().getZooKeeperWatcher(),
-                       admin.getConnection(), TableName.valueOf(MetaDataUtil.getLocalIndexTableName(tableName)), false);
-               }
-               assertEquals(i, regionsOfIndexTable.size());
-            }
-        }
-        return regionsOfUserTable;
-    }
-
     private List<HRegionInfo> splitDuringScan(Connection conn1, String[] strings, HBaseAdmin admin, boolean isReverse)
             throws SQLException, IOException, InterruptedException {
         ResultSet rs;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ReadOnlyIndexFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ReadOnlyIndexFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ReadOnlyIndexFailureIT.java
index 931fcae..a3ad418 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ReadOnlyIndexFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ReadOnlyIndexFailureIT.java
@@ -32,6 +32,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -41,11 +42,13 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
 import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.BaseOwnClusterHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.hbase.index.Indexer;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableType;
@@ -279,6 +282,13 @@ public class ReadOnlyIndexFailureIT extends BaseOwnClusterHBaseManagedTimeIT {
             if (c.getEnvironment().getRegionInfo().getTable().getNameAsString().contains(INDEX_NAME) && FAIL_WRITE) {
                 throw new DoNotRetryIOException();
             }
+            Mutation operation = miniBatchOp.getOperation(0);
+            Set<byte[]> keySet = operation.getFamilyMap().keySet();
+            for(byte[] family: keySet) {
+                if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX) && FAIL_WRITE) {
+                    throw new DoNotRetryIOException();
+                }
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
----------------------------------------------------------------------
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 450cffa..6b980a4 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
@@ -113,7 +113,7 @@ public class ViewIndexIT extends BaseHBaseManagedTimeIT {
     public ViewIndexIT(boolean isNamespaceMapped) {
         this.isNamespaceMapped = isNamespaceMapped;
         this.physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
-        this.viewIndexPhysicalTableName = MetaDataUtil.getLocalIndexTableName(physicalTableName.getNameAsString());
+        this.viewIndexPhysicalTableName = this.physicalTableName.getNameAsString();
     }
 
 
@@ -172,7 +172,7 @@ public class ViewIndexIT extends BaseHBaseManagedTimeIT {
         String sql = "SELECT * FROM " + VIEW_NAME + " WHERE v2 = 100";
         ResultSet rs = conn1.prepareStatement("EXPLAIN " + sql).executeQuery();
         assertEquals(
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_T ['10',-32768,100]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER T ['10',1,100]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
         rs = conn1.prepareStatement(sql).executeQuery();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
index e675dda..558bdcb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
@@ -33,8 +33,11 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.Shadower;
+import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -305,7 +308,11 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
             
             conn.rollback();
             assertDataAndIndexRows(stmt);
-
+            PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
+            if(localIndex) {
+                dropTable(phoenixConn.getQueryServices().getAdmin(), conn, fullTableName1);
+                dropTable(phoenixConn.getQueryServices().getAdmin(), conn, fullTableName2);
+            }
         } finally {
             conn.close();
         }
@@ -440,7 +447,8 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
             assertEquals("x", rs.getString(1));
             assertEquals("yyyy", rs.getString(2));
             assertFalse(rs.next());
-                        
+            PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
+            if(localIndex) dropTable(phoenixConn.getQueryServices().getAdmin(), conn, fullTableName1);
         } finally {
             conn.close();
         }
@@ -503,9 +511,19 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
             assertEquals("x", rs.getString(1));
             assertEquals("a", rs.getString(2));
             assertFalse(rs.next());
-
+            PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
+            if(localIndex) dropTable(phoenixConn.getQueryServices().getAdmin(), conn, fullTableName1);
         } finally {
             conn.close();
         }
     }
+
+    private void dropTable(HBaseAdmin admin, Connection conn, String tableName) throws SQLException, IOException {
+        conn.createStatement().execute("DROP TABLE IF EXISTS "+ tableName);
+        if(admin.tableExists(tableName)) {
+            admin.disableTable(TableName.valueOf(tableName));
+            admin.deleteTable(TableName.valueOf(tableName));
+        } 
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
index 3e1d5ab..f3fb46a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
@@ -76,7 +76,7 @@ public class RollbackIT extends BaseHBaseManagedTimeIT {
     public static Collection<Boolean[]> data() {
         return Arrays.asList(new Boolean[][] {     
                  { false, false }, { false, true },
-                 { true, false }, { true, true }  
+                 { true, false }, { true, true } 
            });
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
index 8e36bf0..c7364dd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
@@ -152,7 +152,7 @@ public class TxWriteFailureIT extends BaseOwnClusterHBaseManagedTimeIT {
         rs = conn.createStatement().executeQuery("EXPLAIN "+indexSql);
         if(localIndex) {
             assertEquals(
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + DATA_TABLE_FULL_NAME + " [-32768]\n" + 
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + DATA_TABLE_FULL_NAME + " [1]\n" + 
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "CLIENT MERGE SORT",
                 QueryUtil.getExplainPlan(rs));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancerIT.java
deleted file mode 100644
index 1bec821..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancerIT.java
+++ /dev/null
@@ -1,485 +0,0 @@
-/**
- * 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.hbase.index.balancer;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.LoadBalancer;
-import org.apache.hadoop.hbase.master.RegionStates;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.TestSplitTransactionOnCluster.MockedRegionObserver;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
-import org.apache.phoenix.hbase.index.IndexTestingUtils;
-import org.apache.phoenix.hbase.index.Indexer;
-import org.apache.phoenix.hbase.index.master.IndexMasterObserver;
-import org.apache.phoenix.util.ConfigUtil;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.Timeout;
-
-@Category(NeedsOwnMiniClusterTest.class)
-public class IndexLoadBalancerIT {
-
-    private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-    private static HBaseAdmin admin = null;
-    
-    @Rule
-    public Timeout timeout = new Timeout(300, TimeUnit.SECONDS);
-
-    @BeforeClass
-    public static void setupCluster() throws Exception {
-        final int NUM_RS = 4;
-        Configuration conf = UTIL.getConfiguration();
-        conf.setBoolean(HConstants.REGIONSERVER_INFO_PORT_AUTO, true);
-        conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, IndexMasterObserver.class.getName());
-        conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, IndexLoadBalancer.class,
-            LoadBalancer.class);
-        IndexTestingUtils.setupConfig(conf);
-        // disable version checking, so we can test against whatever version of HBase happens to be
-        // installed (right now, its generally going to be SNAPSHOT versions).
-        conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
-        // set replication required parameter
-        ConfigUtil.setReplicationConfigIfAbsent(conf);
-        UTIL.startMiniCluster(NUM_RS);
-        admin = UTIL.getHBaseAdmin();
-    }
-    
-    @Test
-    public void testRoundRobinAssignmentDuringIndexTableCreation() throws Exception {
-        MiniHBaseCluster cluster = UTIL.getHBaseCluster();
-        HMaster master = cluster.getMaster();
-        TableName tableName = TableName.valueOf("testRoundRobinAssignmentDuringIndexTableCreation");
-        TableName indexTableName =
-                TableName.valueOf("testRoundRobinAssignmentDuringIndexTableCreation_index");
-        createUserAndIndexTable(tableName, indexTableName);
-        boolean isRegionColocated =
-                checkForColocation(master, tableName.getNameAsString(), indexTableName
-                        .getNameAsString());
-        assertTrue("User regions and index regions should colocate.", isRegionColocated);
-    }
-    
-    @Test
-    public void testColocationAfterSplit() throws Exception {
-        MiniHBaseCluster cluster = UTIL.getHBaseCluster();
-        HMaster master = cluster.getMaster();
-        // Table names to make use of the
-        TableName tableName = TableName.valueOf("testSplitHooksBeforeAndAfterPONR_1");
-        TableName indexTableName = TableName.valueOf("testSplitHooksBeforeAndAfterPONR_2");
-        HTableDescriptor htd = new HTableDescriptor(tableName);
-        htd.addCoprocessor(MockedRegionObserver.class.getName());
-        htd.addFamily(new HColumnDescriptor("cf"));
-        char c = 'A';
-        byte[][] split = new byte[20][];
-        for (int i = 0; i < 20; i++) {
-            byte[] b = { (byte) c };
-            split[i] = b;
-            c++;
-        }
-        admin.createTable(htd, split);
-        HTableDescriptor iHtd = new HTableDescriptor(indexTableName);
-        iHtd.addFamily(new HColumnDescriptor("cf"));
-        iHtd.setValue(IndexLoadBalancer.PARENT_TABLE_KEY, tableName.toBytes());
-        admin.createTable(iHtd, split);
-
-        // test put with the indexed column
-
-        insertData(tableName);
-        insertData(indexTableName);
-
-        admin.split(tableName.getNameAsString(), "c");
-        List<HRegionInfo> regionsOfUserTable =
-                master.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName);
-
-        while (regionsOfUserTable.size() != 22) {
-            Thread.sleep(100);
-            regionsOfUserTable =
-                    master.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName);
-        }
-
-        List<HRegionInfo> regionsOfIndexTable =
-                master.getAssignmentManager().getRegionStates().getRegionsOfTable(indexTableName);
-
-        while (regionsOfIndexTable.size() != 22) {
-            Thread.sleep(100);
-            regionsOfIndexTable =
-                    master.getAssignmentManager().getRegionStates().getRegionsOfTable(
-                        indexTableName);
-        }
-        boolean isRegionColocated =
-                checkForColocation(master, tableName.getNameAsString(), indexTableName
-                        .getNameAsString());
-        assertTrue("User regions and index regions should colocate.", isRegionColocated);
-    }
-    
-    @Test
-    public void testColocationAfterRegionsMerge() throws Exception {
-        MiniHBaseCluster cluster = UTIL.getHBaseCluster();
-        HMaster master = cluster.getMaster();
-        RegionStates regionStates = master.getAssignmentManager().getRegionStates();
-        // Table names to make use of the
-        TableName tableName = TableName.valueOf("testColocationAfterRegionsMerge");
-        TableName indexTableName = TableName.valueOf("testColocationAfterRegionsMerge_index");
-        createUserAndIndexTable(tableName, indexTableName);
-        ServerName server = cluster.getRegionServer(0).getServerName();
-        List<HRegionInfo> regionsOfUserTable = regionStates.getRegionsOfTable(tableName);
-        Pair<HRegionInfo, HRegionInfo> regionsToMerge = new Pair<HRegionInfo, HRegionInfo>();
-        byte[] startKey1 = { (byte) 'C' };
-        byte[] startKey2 = { (byte) 'D' };
-        for (HRegionInfo region : regionsOfUserTable) {
-            if (Bytes.compareTo(startKey1, region.getStartKey()) == 0) {
-                regionsToMerge.setFirst(region);
-            } else if (Bytes.compareTo(startKey2, region.getStartKey()) == 0) {
-                regionsToMerge.setSecond(region);
-            }
-        }
-        admin.move(regionsToMerge.getFirst().getEncodedNameAsBytes(), Bytes.toBytes(server
-                .toString()));
-        admin.move(regionsToMerge.getSecond().getEncodedNameAsBytes(), Bytes.toBytes(server
-                .toString()));
-
-        List<HRegionInfo> regionsOfIndexTable = regionStates.getRegionsOfTable(indexTableName);
-        Pair<HRegionInfo, HRegionInfo> indexRegionsToMerge = new Pair<HRegionInfo, HRegionInfo>();
-        for (HRegionInfo region : regionsOfIndexTable) {
-            if (Bytes.compareTo(startKey1, region.getStartKey()) == 0) {
-                indexRegionsToMerge.setFirst(region);
-            } else if (Bytes.compareTo(startKey2, region.getStartKey()) == 0) {
-                indexRegionsToMerge.setSecond(region);
-            }
-        }
-        admin.move(indexRegionsToMerge.getFirst().getEncodedNameAsBytes(), Bytes.toBytes(server
-                .toString()));
-        admin.move(indexRegionsToMerge.getSecond().getEncodedNameAsBytes(), Bytes.toBytes(server
-                .toString()));
-        while (!regionStates.getRegionServerOfRegion(regionsToMerge.getFirst()).equals(server)
-                || !regionStates.getRegionServerOfRegion(regionsToMerge.getSecond()).equals(server)
-                || !regionStates.getRegionServerOfRegion(indexRegionsToMerge.getFirst()).equals(
-                    server)
-                || !regionStates.getRegionServerOfRegion(indexRegionsToMerge.getSecond()).equals(
-                    server)) {
-            Threads.sleep(1000);
-        }
-        admin.mergeRegions(regionsToMerge.getFirst().getEncodedNameAsBytes(), regionsToMerge
-                .getSecond().getEncodedNameAsBytes(), true);
-        admin.mergeRegions(indexRegionsToMerge.getFirst().getEncodedNameAsBytes(),
-            indexRegionsToMerge.getSecond().getEncodedNameAsBytes(), true);
-
-        while (regionsOfUserTable.size() != 20 || regionsOfIndexTable.size() != 20) {
-            Thread.sleep(100);
-            regionsOfUserTable = regionStates.getRegionsOfTable(tableName);
-            regionsOfIndexTable = regionStates.getRegionsOfTable(indexTableName);
-        }
-        boolean isRegionColocated =
-                checkForColocation(master, tableName.getNameAsString(), indexTableName
-                        .getNameAsString());
-        assertTrue("User regions and index regions should colocate.", isRegionColocated);
-    }
-
-    private void insertData(TableName tableName) throws IOException, InterruptedException {
-        HTable table = new HTable(admin.getConfiguration(), tableName);
-        Put p = new Put("a".getBytes());
-        p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("Val"));
-        p.add("cf".getBytes(), "q2".getBytes(), Bytes.toBytes("ValForCF2"));
-        table.put(p);
-
-        Put p1 = new Put("b".getBytes());
-        p1.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("Val"));
-        p1.add("cf".getBytes(), "q2".getBytes(), Bytes.toBytes("ValForCF2"));
-        table.put(p1);
-
-        Put p2 = new Put("c".getBytes());
-        p2.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("Val"));
-        p2.add("cf".getBytes(), "q2".getBytes(), Bytes.toBytes("ValForCF2"));
-        table.put(p2);
-
-        Put p3 = new Put("c1".getBytes());
-        p3.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("Val"));
-        p3.add("cf".getBytes(), "q2".getBytes(), Bytes.toBytes("ValForCF2"));
-        table.put(p3);
-
-        Put p4 = new Put("d".getBytes());
-        p4.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("Val"));
-        p4.add("cf".getBytes(), "q2".getBytes(), Bytes.toBytes("ValForCF2"));
-        table.put(p4);
-        admin.flush(tableName.getNameAsString());
-    }
-    
-    @Test
-    public void testRandomAssignmentDuringIndexTableEnable() throws Exception {
-        MiniHBaseCluster cluster = UTIL.getHBaseCluster();
-        HMaster master = cluster.getMaster();
-        master.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", false);
-        TableName tableName = TableName.valueOf("testRandomAssignmentDuringIndexTableEnable");
-        TableName indexTableName =
-                TableName.valueOf("testRandomAssignmentDuringIndexTableEnable_index");
-        createUserAndIndexTable(tableName, indexTableName);
-        admin.disableTable(tableName);
-        admin.disableTable(indexTableName);
-        admin.enableTable(tableName);
-        admin.enableTable(indexTableName);
-        boolean isRegionColocated =
-                checkForColocation(master, tableName.getNameAsString(), indexTableName
-                        .getNameAsString());
-        assertTrue("User regions and index regions should colocate.", isRegionColocated);
-
-    }
-    
-    @Test
-    public void testBalanceCluster() throws Exception {
-        MiniHBaseCluster cluster = UTIL.getHBaseCluster();
-        HMaster master = cluster.getMaster();
-        master.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", false);
-        master.getConfiguration().setBoolean("hbase.master.startup.retainassign", false);
-        master.getConfiguration().setBoolean("hbase.master.loadbalance.bytable", false);
-
-        TableName tableName = TableName.valueOf("testBalanceCluster");
-        TableName indexTableName = TableName.valueOf("testBalanceCluster_index");
-        createUserAndIndexTable(tableName, indexTableName);
-        HTableDescriptor htd1 = new HTableDescriptor(TableName.valueOf("testBalanceCluster1"));
-        htd1.addFamily(new HColumnDescriptor("fam1"));
-        char c = 'A';
-        byte[][] split1 = new byte[12][];
-        for (int i = 0; i < 12; i++) {
-            byte[] b = { (byte) c };
-            split1[i] = b;
-            c++;
-        }
-        admin.setBalancerRunning(false, false);
-        admin.createTable(htd1, split1);
-        admin.disableTable(tableName);
-        admin.enableTable(tableName);
-        admin.setBalancerRunning(true, false);
-        admin.balancer();
-        boolean isRegionsColocated =
-                checkForColocation(master, tableName.getNameAsString(), indexTableName
-                        .getNameAsString());
-        assertTrue("User regions and index regions should colocate.", isRegionsColocated);
-    }
-    
-    @Test
-    public void testBalanceByTable() throws Exception {
-        ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(UTIL);
-        MiniHBaseCluster cluster = UTIL.getHBaseCluster();
-        HMaster master = cluster.getMaster();
-        master.getConfiguration().setBoolean("hbase.master.loadbalance.bytable", true);
-        TableName tableName = TableName.valueOf("testBalanceByTable");
-        TableName indexTableName = TableName.valueOf("testBalanceByTable_index");
-        createUserAndIndexTable(tableName, indexTableName);
-        HTableDescriptor htd1 = new HTableDescriptor(TableName.valueOf("testBalanceByTable1"));
-        htd1.addFamily(new HColumnDescriptor("fam1"));
-        char c = 'A';
-        byte[][] split1 = new byte[12][];
-        for (int i = 0; i < 12; i++) {
-            byte[] b = { (byte) c };
-            split1[i] = b;
-            c++;
-        }
-        admin.disableTable(tableName);
-        admin.enableTable(tableName);
-        admin.setBalancerRunning(true, false);
-        boolean isRegionColocated =
-                checkForColocation(master, tableName.getNameAsString(), indexTableName
-                        .getNameAsString());
-        assertTrue("User regions and index regions should colocate.", isRegionColocated);
-        admin.balancer();
-        Thread.sleep(10000);
-        ZKAssign.blockUntilNoRIT(zkw);
-        while (master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
-            Threads.sleep(1000);
-        }
-        isRegionColocated =
-                checkForColocation(master, tableName.getNameAsString(), indexTableName
-                        .getNameAsString());
-        assertTrue("User regions and index regions should colocate.", isRegionColocated);
-    }
-    
-    @Test
-    public void testRoundRobinAssignmentAfterRegionServerDown() throws Exception {
-        ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(UTIL);
-        MiniHBaseCluster cluster = UTIL.getHBaseCluster();
-        HMaster master = cluster.getMaster();
-        TableName tableName = TableName.valueOf("testRoundRobinAssignmentAfterRegionServerDown");
-        TableName indexTableName =
-                TableName.valueOf("testRoundRobinAssignmentAfterRegionServerDown_index");
-        createUserAndIndexTable(tableName, indexTableName);
-        HRegionServer regionServer = cluster.getRegionServer(1);
-        regionServer.abort("Aborting to test random assignment after region server down");
-        while (master.getServerManager().areDeadServersInProgress()) {
-            Thread.sleep(1000);
-        }
-        ZKAssign.blockUntilNoRIT(zkw);
-        while (master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
-            Threads.sleep(1000);
-        }
-        boolean isRegionColocated =
-                checkForColocation(master, tableName.getNameAsString(), indexTableName
-                        .getNameAsString());
-        assertTrue("User regions and index regions should colocate.", isRegionColocated);
-
-    }
-    
-    @Test
-    public void testRetainAssignmentDuringMasterStartUp() throws Exception {
-        ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(UTIL);
-        MiniHBaseCluster cluster = UTIL.getHBaseCluster();
-        HMaster master = cluster.getMaster();
-        master.getConfiguration().setBoolean("hbase.master.startup.retainassign", true);
-        TableName tableName = TableName.valueOf("testRetainAssignmentDuringMasterStartUp");
-        TableName indexTableName =
-                TableName.valueOf("testRetainAssignmentDuringMasterStartUp_index");
-        createUserAndIndexTable(tableName, indexTableName);
-        UTIL.shutdownMiniHBaseCluster();
-        UTIL.startMiniHBaseCluster(1, 4);
-        cluster = UTIL.getHBaseCluster();
-        master = cluster.getMaster();
-        if (admin != null) {
-            admin.close();
-            admin = new HBaseAdmin(master.getConfiguration());
-        }
-        ZKAssign.blockUntilNoRIT(zkw);
-        while (master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
-            Threads.sleep(1000);
-        }
-        boolean isRegionColocated =
-                checkForColocation(master, tableName.getNameAsString(), indexTableName
-                        .getNameAsString());
-        assertTrue("User regions and index regions should colocate.", isRegionColocated);
-
-    }
-    
-    @Ignore // FIXME: PHOENIX-2625 
-    @Test
-    public void testRoundRobinAssignmentDuringMasterStartUp() throws Exception {
-        MiniHBaseCluster cluster = UTIL.getHBaseCluster();
-        HMaster master = cluster.getMaster();
-        UTIL.getConfiguration().setBoolean("hbase.master.startup.retainassign", false);
-
-        TableName tableName = TableName.valueOf("testRoundRobinAssignmentDuringMasterStartUp");
-        TableName indexTableName =
-                TableName.valueOf("testRoundRobinAssignmentDuringMasterStartUp_index");
-        createUserAndIndexTable(tableName, indexTableName);
-        UTIL.shutdownMiniHBaseCluster();
-        cluster.waitUntilShutDown();
-        UTIL.startMiniHBaseCluster(1, 4);
-        cluster = UTIL.getHBaseCluster();
-        if (admin != null) {
-            admin.close();
-            admin = new HBaseAdmin(cluster.getMaster().getConfiguration());
-        }
-        master = cluster.getMaster();
-        while (master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
-            Threads.sleep(1000);
-        }
-        boolean isRegionColocated =
-                checkForColocation(master, tableName.getNameAsString(), indexTableName
-                        .getNameAsString());
-        assertTrue("User regions and index regions should colocate.", isRegionColocated);
-    }
-
-    private void createUserAndIndexTable(TableName tableName, TableName indexTableName)
-            throws IOException {
-        HTableDescriptor htd = new HTableDescriptor(tableName);
-        htd.addFamily(new HColumnDescriptor("cf"));
-        char c = 'A';
-        byte[][] split = new byte[20][];
-        for (int i = 0; i < 20; i++) {
-            byte[] b = { (byte) c };
-            split[i] = b;
-            c++;
-        }
-        admin.createTable(htd, split);
-        HTableDescriptor iHtd = new HTableDescriptor(indexTableName);
-        iHtd.addFamily(new HColumnDescriptor("cf"));
-        iHtd.setValue(IndexLoadBalancer.PARENT_TABLE_KEY, tableName.toBytes());
-        admin.createTable(iHtd, split);
-    }
-
-    private List<Pair<byte[], ServerName>> getStartKeysAndLocations(HMaster master, String tableName)
-            throws IOException, InterruptedException {
-
-        List<Pair<HRegionInfo, ServerName>> tableRegionsAndLocations =
-                MetaTableAccessor.getTableRegionsAndLocations(master.getZooKeeper(), master.getConnection(),
-                        TableName.valueOf(tableName));
-        List<Pair<byte[], ServerName>> startKeyAndLocationPairs =
-                new ArrayList<Pair<byte[], ServerName>>(tableRegionsAndLocations.size());
-        Pair<byte[], ServerName> startKeyAndLocation = null;
-        for (Pair<HRegionInfo, ServerName> regionAndLocation : tableRegionsAndLocations) {
-            startKeyAndLocation =
-                    new Pair<byte[], ServerName>(regionAndLocation.getFirst().getStartKey(),
-                            regionAndLocation.getSecond());
-            startKeyAndLocationPairs.add(startKeyAndLocation);
-        }
-        return startKeyAndLocationPairs;
-
-    }
-    
-    public boolean checkForColocation(HMaster master, String tableName, String indexTableName)
-            throws IOException, InterruptedException {
-        List<Pair<byte[], ServerName>> uTableStartKeysAndLocations =
-                getStartKeysAndLocations(master, tableName);
-        List<Pair<byte[], ServerName>> iTableStartKeysAndLocations =
-                getStartKeysAndLocations(master, indexTableName);
-
-        boolean regionsColocated = true;
-        if (uTableStartKeysAndLocations.size() != iTableStartKeysAndLocations.size()) {
-            regionsColocated = false;
-        } else {
-            for (int i = 0; i < uTableStartKeysAndLocations.size(); i++) {
-                Pair<byte[], ServerName> uStartKeyAndLocation = uTableStartKeysAndLocations.get(i);
-                Pair<byte[], ServerName> iStartKeyAndLocation = iTableStartKeysAndLocations.get(i);
-
-                if (Bytes.compareTo(uStartKeyAndLocation.getFirst(), iStartKeyAndLocation
-                        .getFirst()) == 0) {
-                    if (uStartKeyAndLocation.getSecond().equals(iStartKeyAndLocation.getSecond())) {
-                        continue;
-                    }
-                }
-                regionsColocated = false;
-            }
-        }
-        return regionsColocated;
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
index 6cf8fa1..7b5287a 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
@@ -74,6 +75,9 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
         HRegionInfo childRegion = region.getRegionInfo();
         byte[] splitKey = null;
         if (reader == null && r != null) {
+            if(!p.toString().contains(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                return super.preStoreFileReaderOpen(ctx, fs, p, in, size, cacheConf, r, reader);
+            }
             Scan scan = MetaTableAccessor.getScanForTableName(tableName);
             SingleColumnValueFilter scvf = null;
             if (Reference.isTopFileRegion(r.getFileRegion())) {
@@ -140,8 +144,7 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
             try {
                 conn = QueryUtil.getConnectionOnServer(ctx.getEnvironment().getConfiguration()).unwrap(
                             PhoenixConnection.class);
-                String userTableName = MetaDataUtil.getUserTableName(tableName.getNameAsString());
-                PTable dataTable = PhoenixRuntime.getTable(conn, userTableName);
+                PTable dataTable = PhoenixRuntime.getTable(conn, tableName.getNameAsString());
                 List<PTable> indexes = dataTable.getIndexes();
                 Map<ImmutableBytesWritable, IndexMaintainer> indexMaintainers =
                         new HashMap<ImmutableBytesWritable, IndexMaintainer>();
@@ -180,7 +183,11 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
     public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
             Store store, List<? extends KeyValueScanner> scanners, ScanType scanType,
             long earliestPutTs, InternalScanner s, CompactionRequest request) throws IOException {
-        if (!scanType.equals(ScanType.COMPACT_DROP_DELETES) || s != null || !store.hasReferences()) {
+        if (!store.getFamily().getNameAsString()
+                .startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)
+                || !scanType.equals(ScanType.COMPACT_DROP_DELETES)
+                || s != null
+                || !store.hasReferences()) {
             return s;
         }
         List<StoreFileScanner> newScanners = new ArrayList<StoreFileScanner>(scanners.size());
@@ -236,7 +243,9 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
     public KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
         final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
         final KeyValueScanner s) throws IOException {
-        if(store.hasReferences()) {
+        if (store.getFamily().getNameAsString()
+                .startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)
+                && store.hasReferences()) {
             long readPt = c.getEnvironment().getRegion().getReadpoint(scan.getIsolationLevel());
             boolean scanUsePread = c.getEnvironment().getConfiguration().getBoolean("hbase.storescanner.use.pread", scan.isSmall());
             Collection<StoreFile> storeFiles = store.getStorefiles();


[2/5] phoenix git commit: PHOENIX-1734 Local index improvements(Rajeshbabu)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancer.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancer.java
index 146028e..3353f9e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancer.java
@@ -17,677 +17,8 @@
  */
 package org.apache.phoenix.hbase.index.balancer;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.master.LoadBalancer;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.util.EnvironmentEdgeManager;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * <p>This class is an extension of the load balancer class. 
- * It allows to co-locate the regions of the user table and the regions of corresponding
- * index table if any.</p> 
- * 
- * </>roundRobinAssignment, retainAssignment -> index regions will follow the actual table regions. 
- * randomAssignment, balancerCluster -> either index table or actual table region(s) will follow
- * each other based on which ever comes first.</p> 
- * 
- * <p>In case of master failover there is a chance that the znodes of the index
- * table and actual table are left behind. Then in that scenario we may get randomAssignment for
- * either the actual table region first or the index table region first.</p>
- * 
- * <p>In case of balancing by table any table can balance first.</p>
- * 
- */
-
-public class IndexLoadBalancer implements LoadBalancer {
-
-    private static final Log LOG = LogFactory.getLog(IndexLoadBalancer.class);
-
-    public static final byte[] PARENT_TABLE_KEY = Bytes.toBytes("PARENT_TABLE");
-
-    public static final String INDEX_BALANCER_DELEGATOR = "hbase.index.balancer.delegator.class";
-
-    private LoadBalancer delegator;
-
-    private MasterServices master;
-
-    private Configuration conf;
-
-    private ClusterStatus clusterStatus;
-
-    private static final Random RANDOM = new Random(EnvironmentEdgeManager.currentTimeMillis());
-
-    Map<TableName, TableName> userTableVsIndexTable = new HashMap<TableName, TableName>();
-
-    Map<TableName, TableName> indexTableVsUserTable = new HashMap<TableName, TableName>();
-
-    /**
-     * Maintains colocation information of user regions and corresponding index regions.
-     */
-    private Map<TableName, Map<ImmutableBytesWritable, ServerName>> colocationInfo =
-            new ConcurrentHashMap<TableName, Map<ImmutableBytesWritable, ServerName>>();
-
-    private Set<TableName> balancedTables = new HashSet<TableName>();
-
-    private boolean stopped = false;
-
-    @Override
-    public void initialize() throws HBaseIOException {
-        Class<? extends LoadBalancer> delegatorKlass =
-                conf.getClass(INDEX_BALANCER_DELEGATOR, StochasticLoadBalancer.class,
-                    LoadBalancer.class);
-        this.delegator = ReflectionUtils.newInstance(delegatorKlass, conf);
-        this.delegator.setClusterStatus(clusterStatus);
-        this.delegator.setMasterServices(this.master);
-        this.delegator.initialize();
-        try {
-            populateTablesToColocate(this.master.getTableDescriptors().getAll());
-        } catch (IOException e) {
-            throw new HBaseIOException(e);
-        }
-    }
-
-    @Override
-    public Configuration getConf() {
-        return conf;
-    }
-
-    @Override
-    public void setConf(Configuration configuration) {
-        this.conf = configuration;
-    }
-
-    @Override
-    public void onConfigurationChange(Configuration conf) {
-        setConf(conf);
-    }
-
-    @Override
-    public void setClusterStatus(ClusterStatus st) {
-        this.clusterStatus = st;
-    }
-
-    public Map<TableName, Map<ImmutableBytesWritable, ServerName>> getColocationInfo() {
-        return colocationInfo;
-    }
-
-    @Override
-    public void setMasterServices(MasterServices masterServices) {
-        this.master = masterServices;
-    }
-
-    @Override
-    public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState)
-            throws HBaseIOException {
-        synchronized (this.colocationInfo) {
-            boolean balanceByTable = conf.getBoolean("hbase.master.loadbalance.bytable", false);
-            List<RegionPlan> regionPlans = null;
-
-            TableName tableName = null;
-            if (balanceByTable) {
-                Map<ImmutableBytesWritable, ServerName> tableKeys = null;
-                for (Entry<ServerName, List<HRegionInfo>> serverVsRegionList : clusterState
-                        .entrySet()) {
-                    ServerName sn = serverVsRegionList.getKey();
-                    List<HRegionInfo> regionInfos = serverVsRegionList.getValue();
-                    if (regionInfos.isEmpty()) {
-                        continue;
-                    }
-                    if (!isTableColocated(regionInfos.get(0).getTable())) {
-                        return this.delegator.balanceCluster(clusterState);
-                    }
-                    // Just get the table name from any one of the values in the regioninfo list
-                    if (tableName == null) {
-                        tableName = regionInfos.get(0).getTable();
-                        tableKeys = this.colocationInfo.get(tableName);
-                    }
-                    // Check and modify the colocation info map based on values of cluster state
-                    // because we
-                    // will
-                    // call balancer only when the cluster is in stable and reliable state.
-                    if (tableKeys != null) {
-                        for (HRegionInfo hri : regionInfos) {
-                            updateServer(tableKeys, sn, hri);
-                        }
-                    }
-                }
-                // If user table is already balanced find the index table plans from the user table
-                // plans
-                // or vice verca.
-                TableName mappedTableName = getMappedTableToColocate(tableName);
-                if (balancedTables.contains(mappedTableName)) {
-                    balancedTables.remove(mappedTableName);
-                    regionPlans = new ArrayList<RegionPlan>();
-                    return prepareRegionPlansForClusterState(clusterState, regionPlans);
-                } else {
-                    balancedTables.add(tableName);
-                    regionPlans = this.delegator.balanceCluster(clusterState);
-                    if (regionPlans == null) {
-                        if (LOG.isDebugEnabled()) {
-                            LOG.debug(tableName + " regions already balanced.");
-                        }
-                        return null;
-                    } else {
-                        updateRegionPlans(regionPlans);
-                        return regionPlans;
-                    }
-                }
-
-            } else {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Seperating user tables and index tables regions of "
-                            + "each region server in the cluster.");
-                }
-                Map<ServerName, List<HRegionInfo>> userClusterState =
-                        new HashMap<ServerName, List<HRegionInfo>>();
-                Map<ServerName, List<HRegionInfo>> indexClusterState =
-                        new HashMap<ServerName, List<HRegionInfo>>();
-                for (Entry<ServerName, List<HRegionInfo>> serverVsRegionList : clusterState
-                        .entrySet()) {
-                    ServerName sn = serverVsRegionList.getKey();
-                    List<HRegionInfo> regionsInfos = serverVsRegionList.getValue();
-                    List<HRegionInfo> idxRegionsToBeMoved = new ArrayList<HRegionInfo>();
-                    List<HRegionInfo> userRegionsToBeMoved = new ArrayList<HRegionInfo>();
-                    for (HRegionInfo hri : regionsInfos) {
-                        if (hri.isMetaRegion()) {
-                            continue;
-                        }
-                        tableName = hri.getTable();
-                        // Check and modify the colocation info map based on values of cluster state
-                        // because we
-                        // will
-                        // call balancer only when the cluster is in stable and reliable state.
-                        if (isTableColocated(tableName)) {
-                            // table name may change every time thats why always need to get table
-                            // entries.
-                            Map<ImmutableBytesWritable, ServerName> tableKeys =
-                                    this.colocationInfo.get(tableName);
-                            if (tableKeys != null) {
-                                updateServer(tableKeys, sn, hri);
-                            }
-                        }
-                        if (indexTableVsUserTable.containsKey(tableName)) {
-                            idxRegionsToBeMoved.add(hri);
-                            continue;
-                        }
-                        userRegionsToBeMoved.add(hri);
-                    }
-                    // there may be dummy entries here if assignments by table is set
-                    userClusterState.put(sn, userRegionsToBeMoved);
-                    indexClusterState.put(sn, idxRegionsToBeMoved);
-                }
-
-                regionPlans = this.delegator.balanceCluster(userClusterState);
-                if (regionPlans == null) {
-                    if (LOG.isDebugEnabled()) {
-                        LOG.debug("User region plan is null.");
-                    }
-                    regionPlans = new ArrayList<RegionPlan>();
-                } else {
-                    updateRegionPlans(regionPlans);
-                }
-                return prepareRegionPlansForClusterState(indexClusterState, regionPlans);
-            }
-        }
-    }
-
-    private void updateServer(Map<ImmutableBytesWritable, ServerName> tableKeys, ServerName sn,
-            HRegionInfo hri) {
-        ImmutableBytesWritable startKey = new ImmutableBytesWritable(hri.getStartKey());
-        ServerName existingServer = tableKeys.get(startKey);
-        if (!sn.equals(existingServer)) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("There is a mismatch in the existing server name for the region " + hri
-                        + ".  Replacing the server " + existingServer + " with " + sn + ".");
-            }
-            tableKeys.put(startKey, sn);
-        }
-    }
-
-    /**
-     * Prepare region plans for cluster state
-     * @param clusterState if balancing is table wise then cluster state contains only indexed or
-     *            index table regions, otherwise it contains all index tables regions.
-     * @param regionPlans
-     * @return
-     */
-    private List<RegionPlan> prepareRegionPlansForClusterState(
-            Map<ServerName, List<HRegionInfo>> clusterState, List<RegionPlan> regionPlans) {
-        if (regionPlans == null) regionPlans = new ArrayList<RegionPlan>();
-        ImmutableBytesWritable startKey = new ImmutableBytesWritable();
-        for (Entry<ServerName, List<HRegionInfo>> serverVsRegionList : clusterState.entrySet()) {
-            List<HRegionInfo> regionInfos = serverVsRegionList.getValue();
-            ServerName server = serverVsRegionList.getKey();
-            for (HRegionInfo regionInfo : regionInfos) {
-                if (!isTableColocated(regionInfo.getTable())) continue;
-                TableName mappedTableName = getMappedTableToColocate(regionInfo.getTable());
-                startKey.set(regionInfo.getStartKey());
-                ServerName sn = this.colocationInfo.get(mappedTableName).get(startKey);
-                if (sn.equals(server)) {
-                    continue;
-                } else {
-                    RegionPlan rp = new RegionPlan(regionInfo, server, sn);
-                    if (LOG.isDebugEnabled()) {
-                        LOG.debug("Selected server " + rp.getDestination()
-                                + " as destination for region "
-                                + regionInfo.getRegionNameAsString() + " from colocation info.");
-                    }
-                    regionOnline(regionInfo, rp.getDestination());
-                    regionPlans.add(rp);
-                }
-            }
-        }
-        return regionPlans;
-    }
-
-    private void updateRegionPlans(List<RegionPlan> regionPlans) {
-        for (RegionPlan regionPlan : regionPlans) {
-            HRegionInfo hri = regionPlan.getRegionInfo();
-            if (!isTableColocated(hri.getTable())) continue;
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Saving region plan of region " + hri.getRegionNameAsString() + '.');
-            }
-            regionOnline(hri, regionPlan.getDestination());
-        }
-    }
-
-    @Override
-    public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(List<HRegionInfo> regions,
-            List<ServerName> servers) throws HBaseIOException {
-        List<HRegionInfo> userRegions = new ArrayList<HRegionInfo>();
-        List<HRegionInfo> indexRegions = new ArrayList<HRegionInfo>();
-        for (HRegionInfo hri : regions) {
-            seperateUserAndIndexRegion(hri, userRegions, indexRegions);
-        }
-        Map<ServerName, List<HRegionInfo>> bulkPlan = null;
-        if (!userRegions.isEmpty()) {
-            bulkPlan = this.delegator.roundRobinAssignment(userRegions, servers);
-            // This should not happen.
-            if (null == bulkPlan) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("No region plans selected for user regions in roundRobinAssignment.");
-                }
-                return null;
-            }
-            savePlan(bulkPlan);
-        }
-        bulkPlan = prepareIndexRegionsPlan(indexRegions, bulkPlan, servers);
-        return bulkPlan;
-    }
-
-    private void seperateUserAndIndexRegion(HRegionInfo hri, List<HRegionInfo> userRegions,
-            List<HRegionInfo> indexRegions) {
-        if (indexTableVsUserTable.containsKey(hri.getTable())) {
-            indexRegions.add(hri);
-            return;
-        }
-        userRegions.add(hri);
-    }
-
-    private Map<ServerName, List<HRegionInfo>> prepareIndexRegionsPlan(
-            List<HRegionInfo> indexRegions, Map<ServerName, List<HRegionInfo>> bulkPlan,
-            List<ServerName> servers) throws HBaseIOException {
-        if (null != indexRegions && !indexRegions.isEmpty()) {
-            if (null == bulkPlan) {
-                bulkPlan = new ConcurrentHashMap<ServerName, List<HRegionInfo>>();
-            }
-            for (HRegionInfo hri : indexRegions) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Preparing region plan for index region "
-                            + hri.getRegionNameAsString() + '.');
-                }
-                ServerName destServer = getDestServerForIdxRegion(hri);
-                List<HRegionInfo> destServerRegions = null;
-                if (destServer == null) destServer = this.randomAssignment(hri, servers);
-                if (destServer != null) {
-                    destServerRegions = bulkPlan.get(destServer);
-                    if (null == destServerRegions) {
-                        destServerRegions = new ArrayList<HRegionInfo>();
-                        bulkPlan.put(destServer, destServerRegions);
-                    }
-                    if (LOG.isDebugEnabled()) {
-                        LOG.debug("Server " + destServer + " selected for region "
-                                + hri.getRegionNameAsString() + '.');
-                    }
-                    destServerRegions.add(hri);
-                    regionOnline(hri, destServer);
-                }
-            }
-        }
-        return bulkPlan;
-    }
-
-    private ServerName getDestServerForIdxRegion(HRegionInfo hri) {
-        // Every time we calculate the table name because in case of master restart the index
-        // regions
-        // may be coming for different index tables.
-        TableName actualTable = getMappedTableToColocate(hri.getTable());
-        ImmutableBytesWritable startkey = new ImmutableBytesWritable(hri.getStartKey());
-        synchronized (this.colocationInfo) {
-
-            Map<ImmutableBytesWritable, ServerName> tableKeys = colocationInfo.get(actualTable);
-            if (null == tableKeys) {
-                // Can this case come
-                return null;
-            }
-            if (tableKeys.containsKey(startkey)) {
-                // put index region location if corresponding user region found in regionLocation
-                // map.
-                ServerName sn = tableKeys.get(startkey);
-                regionOnline(hri, sn);
-                return sn;
-            }
-        }
-        return null;
-    }
-
-    private void savePlan(Map<ServerName, List<HRegionInfo>> bulkPlan) {
-        synchronized (this.colocationInfo) {
-            for (Entry<ServerName, List<HRegionInfo>> e : bulkPlan.entrySet()) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Saving user regions' plans for server " + e.getKey() + '.');
-                }
-                for (HRegionInfo hri : e.getValue()) {
-                    if (!isTableColocated(hri.getTable())) continue;
-                    regionOnline(hri, e.getKey());
-                }
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Saved user regions' plans for server " + e.getKey() + '.');
-                }
-            }
-        }
-    }
-
-    @Override
-    public Map<ServerName, List<HRegionInfo>> retainAssignment(
-            Map<HRegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
-        Map<HRegionInfo, ServerName> userRegionsMap =
-                new ConcurrentHashMap<HRegionInfo, ServerName>();
-        List<HRegionInfo> indexRegions = new ArrayList<HRegionInfo>();
-        for (Entry<HRegionInfo, ServerName> e : regions.entrySet()) {
-            seperateUserAndIndexRegion(e, userRegionsMap, indexRegions, servers);
-        }
-        Map<ServerName, List<HRegionInfo>> bulkPlan = null;
-        if (!userRegionsMap.isEmpty()) {
-            bulkPlan = this.delegator.retainAssignment(userRegionsMap, servers);
-            if (bulkPlan == null) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Empty region plan for user regions.");
-                }
-                return null;
-            }
-            savePlan(bulkPlan);
-        }
-        bulkPlan = prepareIndexRegionsPlan(indexRegions, bulkPlan, servers);
-        return bulkPlan;
-    }
-
-    private void seperateUserAndIndexRegion(Entry<HRegionInfo, ServerName> e,
-            Map<HRegionInfo, ServerName> userRegionsMap, List<HRegionInfo> indexRegions,
-            List<ServerName> servers) {
-        HRegionInfo hri = e.getKey();
-        if (indexTableVsUserTable.containsKey(hri.getTable())) {
-            indexRegions.add(hri);
-            return;
-        }
-        if (e.getValue() == null) {
-            userRegionsMap.put(hri, servers.get(RANDOM.nextInt(servers.size())));
-        } else {
-            userRegionsMap.put(hri, e.getValue());
-        }
-    }
-
-    @Override
-    public Map<HRegionInfo, ServerName> immediateAssignment(List<HRegionInfo> regions,
-            List<ServerName> servers) throws HBaseIOException {
-        return this.delegator.immediateAssignment(regions, servers);
-    }
-
-    @Override
-    public ServerName randomAssignment(HRegionInfo regionInfo, List<ServerName> servers)
-            throws HBaseIOException {
-        if (!isTableColocated(regionInfo.getTable())) {
-            return this.delegator.randomAssignment(regionInfo, servers);
-        }
-        ServerName sn = getServerNameFromMap(regionInfo, servers);
-        if (sn == null) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("No server found for region " + regionInfo.getRegionNameAsString() + '.');
-            }
-            sn = getRandomServer(regionInfo, servers);
-        }
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Destination server for region " + regionInfo.getRegionNameAsString()
-                    + " is " + ((sn == null) ? "null" : sn.toString()) + '.');
-        }
-        return sn;
-    }
-
-    private ServerName getRandomServer(HRegionInfo regionInfo, List<ServerName> servers)
-            throws HBaseIOException {
-        ServerName sn = null;
-        sn = this.delegator.randomAssignment(regionInfo, servers);
-        if (sn == null) return null;
-        regionOnline(regionInfo, sn);
-        return sn;
-    }
-
-    private ServerName getServerNameFromMap(HRegionInfo regionInfo, List<ServerName> onlineServers) {
-        TableName tableName = regionInfo.getTable();
-        TableName mappedTable = getMappedTableToColocate(regionInfo.getTable());
-        ImmutableBytesWritable startKey = new ImmutableBytesWritable(regionInfo.getStartKey());
-        synchronized (this.colocationInfo) {
-            Map<ImmutableBytesWritable, ServerName> correspondingTableKeys =
-                    this.colocationInfo.get(mappedTable);
-            Map<ImmutableBytesWritable, ServerName> actualTableKeys =
-                    this.colocationInfo.get(tableName);
-
-            if (null != correspondingTableKeys) {
-                if (correspondingTableKeys.containsKey(startKey)) {
-                    ServerName previousServer = null;
-                    if (null != actualTableKeys) {
-                        previousServer = actualTableKeys.get(startKey);
-                    }
-                    ServerName sn = correspondingTableKeys.get(startKey);
-                    if (null != previousServer) {
-                        // if servername of index region and user region are same in colocationInfo
-                        // clean
-                        // previous plans and return null
-                        if (previousServer.equals(sn)) {
-                            correspondingTableKeys.remove(startKey);
-                            actualTableKeys.remove(startKey);
-                            if (LOG.isDebugEnabled()) {
-                                LOG
-                                        .debug("Both user region plan and corresponding index region plan "
-                                                + "in colocation info are same. Hence clearing the plans to select new plan"
-                                                + " for the region "
-                                                + regionInfo.getRegionNameAsString() + ".");
-                            }
-                            return null;
-                        }
-                    }
-                    if (sn != null && onlineServers.contains(sn)) {
-                        if (LOG.isDebugEnabled()) {
-                            LOG.debug("Updating the region plan of the region "
-                                    + regionInfo.getRegionNameAsString() + " with server " + sn);
-                        }
-                        regionOnline(regionInfo, sn);
-                        return sn;
-                    } else if (sn != null) {
-                        if (LOG.isDebugEnabled()) {
-                            LOG.debug("The location " + sn + " of region with start key"
-                                    + Bytes.toStringBinary(regionInfo.getStartKey())
-                                    + " is not in online. Selecting other region server.");
-                        }
-                        return null;
-                    }
-                }
-            } else {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("No region plans in colocationInfo for table " + mappedTable);
-                }
-            }
-            return null;
-        }
-    }
-
-    @Override
-    public void regionOnline(HRegionInfo regionInfo, ServerName sn) {
-        TableName tableName = regionInfo.getTable();
-        synchronized (this.colocationInfo) {
-            Map<ImmutableBytesWritable, ServerName> tabkeKeys = this.colocationInfo.get(tableName);
-            if (tabkeKeys == null) {
-                tabkeKeys = new ConcurrentHashMap<ImmutableBytesWritable, ServerName>();
-                this.colocationInfo.put(tableName, tabkeKeys);
-            }
-            tabkeKeys.put(new ImmutableBytesWritable(regionInfo.getStartKey()), sn);
-        }
-    }
-
-    public void clearTableRegionPlans(TableName tableName) {
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Clearing regions plans from colocationInfo for table " + tableName);
-        }
-        synchronized (this.colocationInfo) {
-            this.colocationInfo.remove(tableName);
-        }
-    }
-
-    @Override
-    public void regionOffline(HRegionInfo regionInfo) {
-        TableName tableName = regionInfo.getTable();
-        synchronized (this.colocationInfo) {
-            Map<ImmutableBytesWritable, ServerName> tableKeys = this.colocationInfo.get(tableName);
-            if (null == tableKeys) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("No regions of table " + tableName + " in the colocationInfo.");
-                }
-            } else {
-                tableKeys.remove(new ImmutableBytesWritable(regionInfo.getStartKey()));
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("The regioninfo " + regionInfo + " removed from the colocationInfo");
-                }
-            }
-        }
-    }
-
-    @Override
-    public boolean isStopped() {
-        return stopped;
-    }
-
-    @Override
-    public void stop(String why) {
-        LOG.info("Load Balancer stop requested: " + why);
-        stopped = true;
-    }
-
-    public void populateTablesToColocate(Map<String, HTableDescriptor> tableDescriptors) {
-        HTableDescriptor desc = null;
-        for (Entry<String, HTableDescriptor> entry : tableDescriptors.entrySet()) {
-            desc = entry.getValue();
-            if (desc.getValue(PARENT_TABLE_KEY) != null) {
-                addTablesToColocate(TableName.valueOf(desc.getValue(PARENT_TABLE_KEY)), desc
-                        .getTableName());
-            }
-        }
-    }
-
-    /**
-     * Add tables whose regions to co-locate.
-     * @param userTable
-     * @param indexTable
-     */
-    public void addTablesToColocate(TableName userTable, TableName indexTable) {
-        if (userTable.equals(indexTable)) {
-            throw new IllegalArgumentException("Tables to colocate should not be same.");
-        } else if (isTableColocated(userTable)) {
-            throw new IllegalArgumentException("User table already colocated with table "
-                    + getMappedTableToColocate(userTable));
-        } else if (isTableColocated(indexTable)) {
-            throw new IllegalArgumentException("Index table is already colocated with table "
-                    + getMappedTableToColocate(indexTable));
-        }
-        userTableVsIndexTable.put(userTable, indexTable);
-        indexTableVsUserTable.put(indexTable, userTable);
-    }
-
-    /**
-     * Removes the specified table and corresponding table from co-location.
-     * @param table
-     */
-    public void removeTablesFromColocation(TableName table) {
-        TableName other = userTableVsIndexTable.remove(table);
-        if (other != null) {
-            indexTableVsUserTable.remove(other);
-        } else {
-            other = indexTableVsUserTable.remove(table);
-            if (other != null) userTableVsIndexTable.remove(other);
-        }
-    }
-
-    /**
-     * Return mapped table to co-locate.
-     * @param tableName
-     * @return index table if the specified table is user table or vice versa.
-     */
-    public TableName getMappedTableToColocate(TableName tableName) {
-        TableName other = userTableVsIndexTable.get(tableName);
-        return other == null ? indexTableVsUserTable.get(tableName) : other;
-    }
 
-    public boolean isTableColocated(TableName table) {
-        return userTableVsIndexTable.containsKey(table) || indexTableVsUserTable.containsKey(table);
-    }
+public class IndexLoadBalancer extends StochasticLoadBalancer {
 
-    /**
-     * Populates table's region locations into co-location info from master.
-     * @param table
-     */
-    public void populateRegionLocations(TableName table) {
-        synchronized (this.colocationInfo) {
-            if (!isTableColocated(table)) {
-                throw new IllegalArgumentException("Specified table " + table
-                        + " should be in one of the tables to co-locate.");
-            }
-            RegionStates regionStates = this.master.getAssignmentManager().getRegionStates();
-            List<HRegionInfo> onlineRegions = regionStates.getRegionsOfTable(table);
-            for (HRegionInfo hri : onlineRegions) {
-                regionOnline(hri, regionStates.getRegionServerOfRegion(hri));
-            }
-            Map<String, RegionState> regionsInTransition = regionStates.getRegionsInTransition();
-            for (RegionState regionState : regionsInTransition.values()) {
-                if (table.equals(regionState.getRegion().getTable())
-                        && regionState.getServerName() != null) {
-                    regionOnline(regionState.getRegion(), regionState.getServerName());
-                }
-            }
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/master/IndexMasterObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/master/IndexMasterObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/master/IndexMasterObserver.java
index a014da2..2f83f8d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/master/IndexMasterObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/master/IndexMasterObserver.java
@@ -17,98 +17,12 @@
  */
 package org.apache.phoenix.hbase.index.master;
 
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver;
-import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.LoadBalancer;
-import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.master.RegionStates;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.hbase.index.balancer.IndexLoadBalancer;
-import org.apache.phoenix.util.MetaDataUtil;
 
 /**
  * Defines of coprocessor hooks(to support secondary indexing) of operations on
  * {@link org.apache.hadoop.hbase.master.HMaster} process.
  */
 public class IndexMasterObserver extends BaseMasterObserver {
-    IndexLoadBalancer balancer = null;
-
-    @Override
-    public void preMasterInitialization(ObserverContext<MasterCoprocessorEnvironment> ctx)
-            throws IOException {
-        LoadBalancer loadBalancer =
-                ctx.getEnvironment().getMasterServices().getAssignmentManager().getBalancer();
-        if (loadBalancer instanceof IndexLoadBalancer) {
-            balancer = (IndexLoadBalancer) loadBalancer;
-        }
-        super.preMasterInitialization(ctx);
-    }
-
-    @Override
-    public void preCreateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-            HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
-        TableName userTableName = null;
-        if (balancer != null && desc.getValue(IndexLoadBalancer.PARENT_TABLE_KEY) != null) {
-            userTableName =
-                    TableName.valueOf(desc.getValue(IndexLoadBalancer.PARENT_TABLE_KEY));
-            balancer.addTablesToColocate(userTableName, desc.getTableName());
-        }
-        if (userTableName != null) balancer.populateRegionLocations(userTableName);
-        super.preCreateTableHandler(ctx, desc, regions);
-    }
-
-    @Override
-    public void preModifyTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-            TableName tableName, HTableDescriptor htd) throws IOException {
-        HTableDescriptor oldDesc =
-                ctx.getEnvironment().getMasterServices().getTableDescriptors().get(tableName);
-        if (oldDesc.getValue(IndexLoadBalancer.PARENT_TABLE_KEY) == null
-                && htd.getValue(IndexLoadBalancer.PARENT_TABLE_KEY) != null) {
-            TableName userTableName =
-                    TableName.valueOf(htd.getValue(IndexLoadBalancer.PARENT_TABLE_KEY));
-            balancer.addTablesToColocate(userTableName, htd.getTableName());
-        }
-        super.preModifyTableHandler(ctx, tableName, htd);
-    }
-
-    @Override
-    public void postMove(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo region,
-            ServerName srcServer, ServerName destServer) throws IOException {
-        if (balancer != null && balancer.isTableColocated(region.getTable())) {
-            AssignmentManager am = ctx.getEnvironment().getMasterServices().getAssignmentManager();
-            RegionStates regionStates = am.getRegionStates();
-            String tableName = region.getTable().getNameAsString();
-            String correspondingTable = MetaDataUtil.isLocalIndex(region.getTable().getNameAsString())
-                    ? MetaDataUtil.getUserTableName(tableName)
-                    : Bytes.toString(MetaDataUtil.getLocalIndexPhysicalName(tableName.getBytes()));
-            List<HRegionInfo> regions =
-                    regionStates.getRegionsOfTable(TableName.valueOf(correspondingTable));
-            for (HRegionInfo hri : regions) {
-                if (Bytes.compareTo(region.getStartKey(), hri.getStartKey()) == 0
-                        && destServer != null) {
-                    balancer.regionOnline(hri, destServer);
-                    am.addPlan(hri.getEncodedName(), new RegionPlan(hri, null, destServer));
-                    am.unassign(hri);
-                }
-            }
-        }
-        super.postMove(ctx, region, srcServer, destServer);
-    }
 
-    @Override
-    public void postDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-            TableName tableName) throws IOException {
-        if (balancer != null && balancer.isTableColocated(tableName)) {
-            balancer.removeTablesFromColocation(tableName);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexCommitter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexCommitter.java
index d7fef5e..5e3f3ed 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexCommitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexCommitter.java
@@ -32,6 +32,6 @@ public interface IndexCommitter extends Stoppable {
 
   void setup(IndexWriter parent, RegionCoprocessorEnvironment env, String name);
 
-  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
+  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite, boolean allowLocalUpdates)
       throws IndexWriteException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
index 30797b2..cbcec3b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
@@ -128,10 +128,11 @@ public class IndexWriter implements Stoppable {
    * @param indexUpdates Updates to write
  * @throws IOException 
    */
-  public void writeAndKillYourselfOnFailure(Collection<Pair<Mutation, byte[]>> indexUpdates) throws IOException  {
+    public void writeAndKillYourselfOnFailure(Collection<Pair<Mutation, byte[]>> indexUpdates,
+            boolean allowLocalUpdates) throws IOException {
     // convert the strings to htableinterfaces to which we can talk and group by TABLE
     Multimap<HTableInterfaceReference, Mutation> toWrite = resolveTableReferences(indexUpdates);
-    writeAndKillYourselfOnFailure(toWrite);
+    writeAndKillYourselfOnFailure(toWrite, allowLocalUpdates);
   }
 
   /**
@@ -139,9 +140,10 @@ public class IndexWriter implements Stoppable {
    * @param toWrite
  * @throws IOException 
    */
-  public void writeAndKillYourselfOnFailure(Multimap<HTableInterfaceReference, Mutation> toWrite) throws IOException {
+    public void writeAndKillYourselfOnFailure(Multimap<HTableInterfaceReference, Mutation> toWrite,
+            boolean allowLocalUpdates) throws IOException {
     try {
-      write(toWrite);
+      write(toWrite, allowLocalUpdates);
       if (LOG.isTraceEnabled()) {
         LOG.trace("Done writing all index updates!\n\t" + toWrite);
       }
@@ -165,21 +167,24 @@ public class IndexWriter implements Stoppable {
    * @throws IndexWriteException if we cannot successfully write to the index. Whether or not we
    *           stop early depends on the {@link IndexCommitter}.
    */
-  public void write(Collection<Pair<Mutation, byte[]>> toWrite) throws IndexWriteException {
-    write(resolveTableReferences(toWrite));
-  }
+    public void write(Collection<Pair<Mutation, byte[]>> toWrite) throws IndexWriteException {
+    	write(resolveTableReferences(toWrite), false);
+    }
+
+    public void write(Collection<Pair<Mutation, byte[]>> toWrite, boolean allowLocalUpdates) throws IndexWriteException {
+    	write(resolveTableReferences(toWrite), allowLocalUpdates);
+    }
 
   /**
    * see {@link #write(Collection)}
    * @param toWrite
    * @throws IndexWriteException
    */
-  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
-      throws IndexWriteException {
-    this.writer.write(toWrite);
+  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite, boolean allowLocalUpdates)
+	      throws IndexWriteException {
+	  this.writer.write(toWrite, allowLocalUpdates);
   }
 
-
   /**
    * Convert the passed index updates to {@link HTableInterfaceReference}s.
    * @param indexUpdates from the index builder

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java
index 233dc57..0dc11bc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java
@@ -21,8 +21,10 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -98,7 +100,7 @@ public class ParallelWriterIndexCommitter implements IndexCommitter {
     }
 
     @Override
-    public void write(Multimap<HTableInterfaceReference, Mutation> toWrite) throws SingleIndexWriteFailureException {
+    public void write(Multimap<HTableInterfaceReference, Mutation> toWrite, final boolean allowLocalUpdates) throws SingleIndexWriteFailureException {
         /*
          * This bit here is a little odd, so let's explain what's going on. Basically, we want to do the writes in
          * parallel to each index table, so each table gets its own task and is submitted to the pool. Where it gets
@@ -116,7 +118,12 @@ public class ParallelWriterIndexCommitter implements IndexCommitter {
             // doing a complete copy over of all the index update for each table.
             final List<Mutation> mutations = kvBuilder.cloneIfNecessary((List<Mutation>)entry.getValue());
             final HTableInterfaceReference tableReference = entry.getKey();
-            final RegionCoprocessorEnvironment env = this.env;
+			if (env != null
+					&& !allowLocalUpdates
+					&& tableReference.getTableName().equals(
+							env.getRegion().getTableDesc().getNameAsString())) {
+				continue;
+			}
             /*
              * Write a batch of index updates to an index table. This operation stops (is cancelable) via two
              * mechanisms: (1) setting aborted or stopped on the IndexWriter or, (2) interrupting the running thread.
@@ -145,29 +152,14 @@ public class ParallelWriterIndexCommitter implements IndexCommitter {
                         LOG.debug("Writing index update:" + mutations + " to table: " + tableReference);
                     }
                     try {
-                        // TODO: Once HBASE-11766 is fixed, reexamine whether this is necessary.
-                        // Also, checking the prefix of the table name to determine if this is a local
-                        // index is pretty hacky. If we're going to keep this, we should revisit that
-                        // as well.
-                        try {
-                            if (MetaDataUtil.isLocalIndex(tableReference.getTableName())) {
-                                Region indexRegion = IndexUtil.getIndexRegion(env);
-                                if (indexRegion != null) {
-                                    throwFailureIfDone();
-                                    indexRegion.batchMutate(mutations.toArray(new Mutation[mutations.size()]),
-                                        HConstants.NO_NONCE, HConstants.NO_NONCE);
-                                    return null;
-                                }
-                            }
-                        } catch (IOException ignord) {
-                            // when it's failed we fall back to the standard & slow way
-                            if (LOG.isDebugEnabled()) {
-                                LOG.debug("indexRegion.batchMutate failed and fall back to HTable.batch(). Got error="
-                                        + ignord);
-                            }
-                        }
+						if (allowLocalUpdates) {
+							for (Mutation m : mutations) {
+								m.setDurability(Durability.SKIP_WAL);
+							}
+						}
                         HTableInterface table = factory.getTable(tableReference.get());
                         throwFailureIfDone();
+                        int i = 0;
                         table.batch(mutations);
                     } catch (SingleIndexWriteFailureException e) {
                         throw e;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
index 14768ac..fec74ca 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -110,7 +111,7 @@ public class TrackingParallelWriterIndexCommitter implements IndexCommitter {
     }
 
     @Override
-    public void write(Multimap<HTableInterfaceReference, Mutation> toWrite) throws MultiIndexWriteFailureException {
+    public void write(Multimap<HTableInterfaceReference, Mutation> toWrite, final boolean allowLocalUpdates) throws MultiIndexWriteFailureException {
         Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = toWrite.asMap().entrySet();
         TaskBatch<Boolean> tasks = new TaskBatch<Boolean>(entries.size());
         List<HTableInterfaceReference> tables = new ArrayList<HTableInterfaceReference>(entries.size());
@@ -121,6 +122,12 @@ public class TrackingParallelWriterIndexCommitter implements IndexCommitter {
             // track each reference so we can get at it easily later, when determing failures
             final HTableInterfaceReference tableReference = entry.getKey();
             final RegionCoprocessorEnvironment env = this.env;
+			if (env != null
+					&& !allowLocalUpdates
+					&& tableReference.getTableName().equals(
+							env.getRegion().getTableDesc().getNameAsString())) {
+				continue;
+			}
             tables.add(tableReference);
 
             /*
@@ -144,33 +151,16 @@ public class TrackingParallelWriterIndexCommitter implements IndexCommitter {
                     try {
                         // this may have been queued, but there was an abort/stop so we try to early exit
                         throwFailureIfDone();
+						if (allowLocalUpdates) {
+							for (Mutation m : mutations) {
+								m.setDurability(Durability.SKIP_WAL);
+							}
+						}
 
                         if (LOG.isDebugEnabled()) {
                             LOG.debug("Writing index update:" + mutations + " to table: " + tableReference);
                         }
 
-                        try {
-                            // TODO: Once HBASE-11766 is fixed, reexamine whether this is necessary.
-                            // Also, checking the prefix of the table name to determine if this is a local
-                            // index is pretty hacky. If we're going to keep this, we should revisit that
-                            // as well.
-                            if (MetaDataUtil.isLocalIndex(tableReference.getTableName())) {
-                                Region indexRegion = IndexUtil.getIndexRegion(env);
-                                if (indexRegion != null) {
-                                    throwFailureIfDone();
-                                    indexRegion.batchMutate(mutations.toArray(new Mutation[mutations.size()]),
-                                        HConstants.NO_NONCE, HConstants.NO_NONCE);
-                                    return Boolean.TRUE;
-                                }
-                            }
-                        } catch (IOException ignord) {
-                            // when it's failed we fall back to the standard & slow way
-                            if (LOG.isDebugEnabled()) {
-                                LOG.debug("indexRegion.batchMutate failed and fall back to HTable.batch(). Got error="
-                                        + ignord);
-                            }
-                        }
-
                         HTableInterface table = factory.getTable(tableReference.get());
                         throwFailureIfDone();
                         table.batch(mutations);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index 1725b11..6a316d9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -274,6 +274,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     // columns required to evaluate all expressions in indexedExpressions (this does not include columns in the data row key)
     private Set<ColumnReference> indexedColumns;
     private Set<ColumnReference> coveredColumns;
+    private Map<ColumnReference, ColumnReference> coveredColumnsMap;
     // columns required to create index row i.e. indexedColumns + coveredColumns  (this does not include columns in the data row key)
     private Set<ColumnReference> allColumns;
     // TODO remove this in the next major release
@@ -363,6 +364,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         this.indexedColumnTypes = Lists.<PDataType>newArrayListWithExpectedSize(nIndexPKColumns-nDataPKColumns);
         this.indexedExpressions = Lists.newArrayListWithExpectedSize(nIndexPKColumns-nDataPKColumns);
         this.coveredColumns = Sets.newLinkedHashSetWithExpectedSize(nIndexColumns-nIndexPKColumns);
+        this.coveredColumnsMap = Maps.newHashMapWithExpectedSize(nIndexColumns-nIndexPKColumns);
         this.nIndexSaltBuckets  = nIndexSaltBuckets == null ? 0 : nIndexSaltBuckets;
         this.dataEmptyKeyValueCF = SchemaUtil.getEmptyColumnFamily(dataTable);
         this.emptyKeyValueCFPtr = SchemaUtil.getEmptyColumnFamilyPtr(index);
@@ -430,6 +432,14 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             for (PColumn indexColumn : family.getColumns()) {
                 PColumn column = IndexUtil.getDataColumn(dataTable, indexColumn.getName().getString());
                 this.coveredColumns.add(new ColumnReference(column.getFamilyName().getBytes(), column.getName().getBytes()));
+                if(isLocalIndex) {
+                    this.coveredColumnsMap.put(
+                        new ColumnReference(column.getFamilyName().getBytes(), column.getName()
+                                .getBytes()),
+                        new ColumnReference(isLocalIndex ? Bytes.toBytes(IndexUtil
+                                .getLocalIndexColumnFamily(column.getFamilyName().getString()))
+                                : column.getFamilyName().getBytes(), column.getName().getBytes()));
+                }
             }
         }
         this.estimatedIndexRowKeyBytes = estimateIndexRowKeyByteSize(indexColByteSize);
@@ -861,7 +871,12 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                     put.setDurability(!indexWALDisabled ? Durability.USE_DEFAULT : Durability.SKIP_WAL);
                 }
                 //this is a little bit of extra work for installations that are running <0.94.14, but that should be rare and is a short-term set of wrappers - it shouldn't kill GC
-                put.add(kvBuilder.buildPut(rowKey, ref.getFamilyWritable(), cq, ts, value));
+                if(this.isLocalIndex) {
+                    ColumnReference columnReference = this.coveredColumnsMap.get(ref);
+					put.add(kvBuilder.buildPut(rowKey, columnReference.getFamilyWritable(), cq, ts, value));
+                } else {
+                    put.add(kvBuilder.buildPut(rowKey, ref.getFamilyWritable(), cq, ts, value));
+                }
             }
         }
         return put;
@@ -949,11 +964,17 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             // If table delete was single version, then index delete should be as well
             if (deleteType == DeleteType.SINGLE_VERSION) {
                 for (ColumnReference ref : getCoverededColumns()) { // FIXME: Keep Set<byte[]> for index CFs?
+                    if(this.isLocalIndex) {
+						ref = this.coveredColumnsMap.get(ref);
+                    }
                     delete.deleteFamilyVersion(ref.getFamily(), ts);
                 }
                 delete.deleteFamilyVersion(emptyCF, ts);
             } else {
                 for (ColumnReference ref : getCoverededColumns()) { // FIXME: Keep Set<byte[]> for index CFs?
+                    if(this.isLocalIndex) {
+						ref = this.coveredColumnsMap.get(ref);
+                    }
                     delete.deleteFamily(ref.getFamily(), ts);
                 }
                 delete.deleteFamily(emptyCF, ts);
@@ -971,11 +992,15 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                         delete = new Delete(indexRowKey);                    
                         delete.setDurability(!indexWALDisabled ? Durability.USE_DEFAULT : Durability.SKIP_WAL);
                     }
+                    ColumnReference columnReference = ref;
+                    if(this.isLocalIndex) {
+                        columnReference = this.coveredColumnsMap.get(ref);
+                    }
                     // If point delete for data table, then use point delete for index as well
                     if (kv.getTypeByte() == KeyValue.Type.Delete.getCode()) {
-                        delete.deleteColumn(ref.getFamily(), IndexUtil.getIndexColumnName(ref.getFamily(), ref.getQualifier()), ts);
+                        delete.deleteColumn(columnReference.getFamily(), IndexUtil.getIndexColumnName(ref.getFamily(), ref.getQualifier()), ts);
                     } else {
-                        delete.deleteColumns(ref.getFamily(), IndexUtil.getIndexColumnName(ref.getFamily(), ref.getQualifier()), ts);
+                        delete.deleteColumns(columnReference.getFamily(), IndexUtil.getIndexColumnName(ref.getFamily(), ref.getQualifier()), ts);
                     }
                 }
             }
@@ -1030,10 +1055,15 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         isLocalIndex = encodedCoveredolumnsAndLocalIndex < 0;
         int nCoveredColumns = Math.abs(encodedCoveredolumnsAndLocalIndex) - 1;
         coveredColumns = Sets.newLinkedHashSetWithExpectedSize(nCoveredColumns);
+        coveredColumnsMap = Maps.newHashMapWithExpectedSize(nCoveredColumns);
         for (int i = 0; i < nCoveredColumns; i++) {
             byte[] cf = Bytes.readByteArray(input);
             byte[] cq = Bytes.readByteArray(input);
-            coveredColumns.add(new ColumnReference(cf,cq));
+            ColumnReference ref = new ColumnReference(cf,cq);
+            coveredColumns.add(ref);
+            if(isLocalIndex) {
+                coveredColumnsMap.put(ref, new ColumnReference(Bytes.toBytes(IndexUtil.getLocalIndexColumnFamily(Bytes.toString(cf))), cq));
+            }
         }
         // Hack to serialize whether the index row key is optimizable
         int len = WritableUtils.readVInt(input);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
index 8ad4d3e..9d2955b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
@@ -71,7 +71,8 @@ public class PhoenixIndexCodec extends BaseIndexCodec {
             Pair<ValueGetter, IndexUpdate> statePair = state.getIndexUpdateState(maintainer.getAllColumns(), metaData.ignoreNewerMutations());
             ValueGetter valueGetter = statePair.getFirst();
             IndexUpdate indexUpdate = statePair.getSecond();
-            indexUpdate.setTable(maintainer.getIndexTableName());
+            indexUpdate.setTable(maintainer.isLocalIndex() ? state.getEnvironment().getRegion()
+                    .getTableDesc().getName() : maintainer.getIndexTableName());
             Put put = maintainer.buildUpdateMutation(KV_BUILDER, valueGetter, ptr, state.getCurrentTimestamp(), env
                     .getRegion().getRegionInfo().getStartKey(), env.getRegion().getRegionInfo().getEndKey());
             indexUpdate.setUpdate(put);
@@ -95,7 +96,8 @@ public class PhoenixIndexCodec extends BaseIndexCodec {
             Pair<ValueGetter, IndexUpdate> statePair = state.getIndexUpdateState(maintainer.getAllColumns(), metaData.ignoreNewerMutations());
             ValueGetter valueGetter = statePair.getFirst();
             IndexUpdate indexUpdate = statePair.getSecond();
-            indexUpdate.setTable(maintainer.getIndexTableName());
+            indexUpdate.setTable(maintainer.isLocalIndex() ? state.getEnvironment().getRegion()
+                    .getTableDesc().getName() : maintainer.getIndexTableName());
             Delete delete = maintainer.buildDeleteMutation(KV_BUILDER, valueGetter, ptr, state.getPendingUpdate(),
                     state.getCurrentTimestamp(), env.getRegion().getRegionInfo().getStartKey(), env.getRegion().getRegionInfo().getEndKey());
             indexUpdate.setUpdate(delete);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
----------------------------------------------------------------------
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 17da04e..d7850ba 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
@@ -138,7 +138,7 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
             }
 
             // its a local index table, so we need to convert it to the index table names we should disable
-            if (MetaDataUtil.isLocalIndex(ref.getTableName())) {
+            if (MetaDataUtil.hasLocalIndexColumnFamily(env.getRegion().getTableDesc())) {
                 for (String tableName : getLocalIndexNames(ref, mutations)) {
                     indexTableNames.put(tableName, minTimeStamp);
                 }
@@ -224,8 +224,7 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
         try {
             conn = QueryUtil.getConnectionOnServer(this.env.getConfiguration()).unwrap(
                     PhoenixConnection.class);
-            String userTableName = MetaDataUtil.getUserTableName(ref.getTableName());
-            PTable dataTable = PhoenixRuntime.getTable(conn, userTableName);
+            PTable dataTable = PhoenixRuntime.getTable(conn, ref.getTableName());
             List<PTable> indexes = dataTable.getIndexes();
             // local index used to get view id from index mutation row key.
             PTable localIndex = null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
index 5d8879c..3d8124c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Result;
@@ -62,6 +63,7 @@ import org.apache.phoenix.hbase.index.covered.TableState;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.hbase.index.covered.update.ColumnTracker;
 import org.apache.phoenix.hbase.index.covered.update.IndexedColumnGroup;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.write.IndexWriter;
 import org.apache.phoenix.query.KeyRange;
@@ -69,6 +71,7 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.trace.TracingUtils;
 import org.apache.phoenix.trace.util.NullSpan;
+import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil;
@@ -165,7 +168,7 @@ public class PhoenixTransactionalIndexer extends BaseRegionObserver {
 
             // no index updates, so we are done
             if (!indexUpdates.isEmpty()) {
-                this.writer.write(indexUpdates);
+                this.writer.write(indexUpdates, true);
             }
         } catch (Throwable t) {
             String msg = "Failed to update index with entries:" + indexUpdates;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
----------------------------------------------------------------------
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 7f403b0..3e0fd99 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
@@ -176,7 +176,7 @@ public abstract class ExplainTable {
         }
     }
 
-    private void appendPKColumnValue(StringBuilder buf, byte[] range, Boolean isNull, int slotIndex) {
+    private void appendPKColumnValue(StringBuilder buf, byte[] range, Boolean isNull, int slotIndex, boolean changeViewIndexId) {
         if (Boolean.TRUE.equals(isNull)) {
             buf.append("null");
             return;
@@ -198,8 +198,14 @@ public abstract class ExplainTable {
             type.coerceBytes(ptr, type, sortOrder, SortOrder.getDefault());
             range = ptr.get();
         }
-        Format formatter = context.getConnection().getFormatter(type);
-        buf.append(type.toStringLiteral(range, formatter));
+        if (changeViewIndexId) {
+            Short s = (Short) type.toObject(range);
+            s = (short) (s + (-Short.MAX_VALUE));
+            buf.append(s.toString());
+        } else {
+            Format formatter = context.getConnection().getFormatter(type);
+            buf.append(type.toStringLiteral(range, formatter));
+        }
     }
     
     private static class RowKeyValueIterator implements Iterator<byte[]> {
@@ -257,6 +263,7 @@ public abstract class ExplainTable {
                 minMaxIterator = new RowKeyValueIterator(schema, minMaxRange.getRange(bound));
             }
         }
+        boolean isLocalIndex = ScanUtil.isLocalIndex(context.getScan());
         boolean forceSkipScan = this.hint.hasHint(Hint.SKIP_SCAN);
         int nRanges = forceSkipScan ? scanRanges.getRanges().size() : scanRanges.getBoundSlotCount();
         for (int i = 0, minPos = 0; minPos < nRanges || minMaxIterator.hasNext(); i++) {
@@ -275,7 +282,13 @@ public abstract class ExplainTable {
                     minMaxIterator = Iterators.emptyIterator();
                 }
             }
-            appendPKColumnValue(buf, b, isNull, i);
+            if (isLocalIndex
+                    && ((context.getConnection().getTenantId() != null && i == 1) || (context
+                            .getConnection().getTenantId() == null && i == 0))) {
+                appendPKColumnValue(buf, b, isNull, i, true);
+            } else {
+                appendPKColumnValue(buf, b, isNull, i, false);
+            }
             buf.append(',');
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/AbstractBulkLoadTool.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/AbstractBulkLoadTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/AbstractBulkLoadTool.java
index 0525de9..41c39a3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/AbstractBulkLoadTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/AbstractBulkLoadTool.java
@@ -395,13 +395,8 @@ public abstract class AbstractBulkLoadTool extends Configured implements Tool {
         PTable table = PhoenixRuntime.getTable(conn, qualifiedTableName);
         List<TargetTableRef> indexTables = new ArrayList<TargetTableRef>();
         for(PTable indexTable : table.getIndexes()){
-            if (indexTable.getIndexType() == PTable.IndexType.LOCAL) {
-                indexTables.add(new TargetTableRef(indexTable.getName().getString(),
-                        Bytes.toString(MetaDataUtil.getLocalIndexPhysicalName(table.getPhysicalName().getBytes()))));
-            } else {
-                indexTables.add(
-                        new TargetTableRef(indexTable.getName().getString(), indexTable.getPhysicalName().getString()));
-            }
+            indexTables.add(new TargetTableRef(indexTable.getName().getString(), indexTable
+                    .getPhysicalName().getString()));
         }
         return indexTables;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
index 6743688..c93a58b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
@@ -206,8 +206,7 @@ public class IndexTool extends Configured implements Tool {
             // computed from the qDataTable name.
             String physicalIndexTable = pindexTable.getPhysicalName().getString();
             if (IndexType.LOCAL.equals(pindexTable.getIndexType())) {
-                physicalIndexTable = Bytes
-                        .toString(MetaDataUtil.getLocalIndexPhysicalName(pdataTable.getPhysicalName().getBytes()));
+                physicalIndexTable = qDataTable;
             }
 
             final PhoenixConnection pConnection = connection.unwrap(PhoenixConnection.class);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
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 f593dd0..c29c0bf 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
@@ -81,7 +81,6 @@ import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
-import org.apache.hadoop.hbase.regionserver.LocalIndexSplitter;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -850,18 +849,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         null, priority, null);
             }
 
-            if (descriptor.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null
-                    && Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(descriptor
-                            .getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
-                if (!descriptor.hasCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName())) {
-                    descriptor.addCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName(),
-                        null, priority, null);
-                }
-            } else {
-                if (!descriptor.hasCoprocessor(LocalIndexSplitter.class.getName())
-                        && !SchemaUtil.isMetaTable(tableName)
-                        && !SchemaUtil.isSequenceTable(tableName)) {
-                    descriptor.addCoprocessor(LocalIndexSplitter.class.getName(), null, priority, null);
+            Set<byte[]> familiesKeys = descriptor.getFamiliesKeys();
+            for(byte[] family: familiesKeys) {
+                if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                    if (!descriptor.hasCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName())) {
+                        descriptor.addCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName(),
+                            null, priority, null);
+                        break;
+                    }
                 }
             }
 
@@ -1071,8 +1066,19 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             } else {
                 if (isMetaTable) {
                     checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
+                } else {
+                    for(Pair<byte[],Map<String,Object>> family: families) {
+                        if ((newDesc.getValue(HTableDescriptor.SPLIT_POLICY)==null || !newDesc.getValue(HTableDescriptor.SPLIT_POLICY).equals(
+                            IndexRegionSplitPolicy.class.getName()))
+                                && Bytes.toString(family.getFirst()).startsWith(
+                                    QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                                   newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
+                                   break;
+                           }
+                    }
                 }
 
+
                 if (!modifyExistingMetaData) {
                     return existingDesc; // Caller already knows that no metadata was changed
                 }
@@ -1303,60 +1309,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
     }
 
-    private void ensureLocalIndexTableCreated(byte[] physicalTableName, Map<String, Object> tableProps,
-            List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, long timestamp,
-            boolean isNamespaceMapped) throws SQLException {
-        PTable table;
-        String parentTableName = SchemaUtil
-                .getParentTableNameFromIndexTable(Bytes.toString(physicalTableName),
-                        MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX)
-                .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
-        try {
-            synchronized (latestMetaDataLock) {
-                throwConnectionClosedIfNullMetaData();
-                table = latestMetaData.getTableRef(new PTableKey(PName.EMPTY_NAME, parentTableName)).getTable();
-                latestMetaDataLock.notifyAll();
-            }
-            if (table.getTimeStamp() >= timestamp) { // Table in cache is newer than client timestamp which shouldn't be the case
-                throw new TableNotFoundException(table.getSchemaName().getString(), table.getTableName().getString());
-            }
-        } catch (TableNotFoundException e) {
-            byte[] schemaName = Bytes.toBytes(SchemaUtil.getSchemaNameFromFullName(parentTableName));
-            byte[] tableName = Bytes.toBytes(SchemaUtil.getTableNameFromFullName(parentTableName));
-            MetaDataMutationResult result = this.getTable(null, schemaName, tableName, HConstants.LATEST_TIMESTAMP, timestamp);
-            table = result.getTable();
-            if (table == null) {
-                throw e;
-            }
-        }
-        ensureLocalIndexTableCreated(physicalTableName, tableProps, families, splits, isNamespaceMapped);
-    }
-
-    private void ensureLocalIndexTableCreated(byte[] physicalTableName, Map<String, Object> tableProps,
-            List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, boolean isNamespaceMapped)
-                    throws SQLException, TableAlreadyExistsException {
-        
-        // If we're not allowing local indexes or the hbase version is too low,
-        // don't create the local index table
-        if (   !this.getProps().getBoolean(QueryServices.ALLOW_LOCAL_INDEX_ATTRIB, QueryServicesOptions.DEFAULT_ALLOW_LOCAL_INDEX) 
-            || !this.supportsFeature(Feature.LOCAL_INDEX)) {
-                    return;
-        }
-        
-        tableProps.put(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_NAME, TRUE_BYTES_AS_STRING);
-        HTableDescriptor desc = ensureTableCreated(physicalTableName, PTableType.TABLE, tableProps, families, splits,
-                true, isNamespaceMapped);
-        if (desc != null) {
-            if (!Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(desc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
-                String fullTableName = Bytes.toString(physicalTableName);
-                throw new TableAlreadyExistsException(
-                        "Unable to create shared physical table for local indexes.",
-                        SchemaUtil.getSchemaNameFromFullName(fullTableName),
-                        SchemaUtil.getTableNameFromFullName(fullTableName));
-            }
-        }
-    }
-
     private boolean ensureViewIndexTableDropped(byte[] physicalTableName, long timestamp) throws SQLException {
         byte[] physicalIndexName = MetaDataUtil.getViewIndexPhysicalName(physicalTableName);
         HTableDescriptor desc = null;
@@ -1385,22 +1337,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     private boolean ensureLocalIndexTableDropped(byte[] physicalTableName, long timestamp) throws SQLException {
-        byte[] physicalIndexName = MetaDataUtil.getLocalIndexPhysicalName(physicalTableName);
         HTableDescriptor desc = null;
         boolean wasDeleted = false;
         try (HBaseAdmin admin = getAdmin()) {
             try {
-                desc = admin.getTableDescriptor(physicalIndexName);
-                if (Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(desc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
-                    this.tableStatsCache.invalidate(new ImmutableBytesPtr(physicalIndexName));
-                    final ReadOnlyProps props = this.getProps();
-                    final boolean dropMetadata = props.getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
-                    if (dropMetadata) {
-                        admin.disableTable(physicalIndexName);
-                        admin.deleteTable(physicalIndexName);
-                        clearTableRegionCache(physicalIndexName);
-                        wasDeleted = true;
+                desc = admin.getTableDescriptor(physicalTableName);
+                this.tableStatsCache.invalidate(new ImmutableBytesPtr(physicalTableName));
+                final ReadOnlyProps props = this.getProps();
+                final boolean dropMetadata = props.getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
+                if (dropMetadata) {
+                    List<String> columnFamiles = new ArrayList<String>();
+                    for(HColumnDescriptor cf : desc.getColumnFamilies()) {
+                        if(cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                            columnFamiles.add(cf.getNameAsString());
+                        }  
                     }
+                    for(String cf: columnFamiles) {
+                        admin.deleteColumn(physicalTableName, cf);
+                    }  
+                    clearTableRegionCache(physicalTableName);
+                    wasDeleted = true;
                 }
             } catch (org.apache.hadoop.hbase.TableNotFoundException ignore) {
                 // Ignore, as we may never have created a view index table
@@ -1424,9 +1380,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] schemaBytes = rowKeyMetadata[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
         byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
         byte[] tableName = physicalTableName != null ? physicalTableName : SchemaUtil.getTableNameAsBytes(schemaBytes, tableBytes);
-        boolean localIndexTable = Boolean.TRUE.equals(tableProps.remove(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_NAME));
-
-        if ((tableType == PTableType.VIEW && physicalTableName != null) || (tableType != PTableType.VIEW && physicalTableName == null)) {
+        boolean localIndexTable = false;
+        for(Pair<byte[], Map<String, Object>> family: families) {
+               if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                       localIndexTable = true;
+                       break;
+               }
+        }
+        if ((tableType == PTableType.VIEW && physicalTableName != null) || (tableType != PTableType.VIEW && (physicalTableName == null || localIndexTable))) {
             // For views this will ensure that metadata already exists
             // For tables and indexes, this will create the metadata if it doesn't already exist
             ensureTableCreated(tableName, tableType, tableProps, families, splits, true, isNamespaceMapped);
@@ -1436,10 +1397,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             // Physical index table created up front for multi tenant
             // TODO: if viewIndexId is Short.MIN_VALUE, then we don't need to attempt to create it
             if (physicalTableName != null) {
-                if (localIndexTable) {
-                    ensureLocalIndexTableCreated(tableName, tableProps, families, splits,
-                            MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
-                } else if (!MetaDataUtil.isMultiTenant(m, kvBuilder, ptr)) {
+                if (!localIndexTable && !MetaDataUtil.isMultiTenant(m, kvBuilder, ptr)) {
                     ensureViewIndexTableCreated(tenantIdBytes.length == 0 ? null : PNameFactory.newName(tenantIdBytes),
                             physicalTableName, MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
                 }
@@ -1561,6 +1519,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 dropTables(result.getTableNamesToDelete());
             }
             invalidateTables(result.getTableNamesToDelete());
+            long timestamp = MetaDataUtil.getClientTimeStamp(tableMetaData);
             if (tableType == PTableType.TABLE) {
                 boolean isNamespaceMapped = result.getTable().isNamespaceMapped();
                 byte[] physicalName;
@@ -1569,7 +1528,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 } else {
                     physicalName = TableName.valueOf(schemaBytes, tableBytes).getName();
                 }
-                long timestamp = MetaDataUtil.getClientTimeStamp(tableMetaData);
                 ensureViewIndexTableDropped(physicalName, timestamp);
                 ensureLocalIndexTableDropped(physicalName, timestamp);
                 tableStatsCache.invalidate(new ImmutableBytesPtr(physicalName));
@@ -2479,6 +2437,19 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 }
 
                                 if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0) {
+                                    Properties props = PropertiesUtil.deepCopy(metaConnection.getClientInfo());
+                                    props.remove(PhoenixRuntime.CURRENT_SCN_ATTRIB);
+                                    props.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
+                                    PhoenixConnection conn =
+                                            new PhoenixConnection(ConnectionQueryServicesImpl.this,
+                                                    metaConnection.getURL(), props, metaConnection
+                                                            .getMetaDataCache());
+                                    try {
+                                        UpgradeUtil.upgradeLocalIndexes(conn, true);
+                                    } finally {
+                                        if (conn != null) conn.close();
+                                    }
+
                                     metaConnection = addColumnsIfNotExists(metaConnection,
                                             PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                             MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 - 2,
@@ -3622,7 +3593,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return supportsFeature(ConnectionQueryServices.Feature.RENEW_LEASE) && renewLeaseEnabled;
     }
 
-
     @Override
     public HRegionLocation getTableRegionLocation(byte[] tableName, byte[] row) throws SQLException {
        /*

http://git-wip-us.apache.org/repos/asf/phoenix/blob/10909ae5/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 4efb708..91c84e0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -166,6 +166,7 @@ public interface QueryConstants {
     public static final byte[] ARRAY_VALUE_COLUMN_QUALIFIER = ByteUtil.EMPTY_BYTE_ARRAY;
 
     public static final byte[] TRUE = new byte[] {1};
+    
 
     /**
      * Separator used between variable length keys for a composite key.
@@ -195,6 +196,16 @@ public interface QueryConstants {
     public static final ImmutableBytesPtr DEFAULT_COLUMN_FAMILY_BYTES_PTR = new ImmutableBytesPtr(
             DEFAULT_COLUMN_FAMILY_BYTES);
 
+    public static final String LOCAL_INDEX_COLUMN_FAMILY_PREFIX = "L#";
+    public static final byte[] LOCAL_INDEX_COLUMN_FAMILY_PREFIX_BYTES = Bytes.toBytes(LOCAL_INDEX_COLUMN_FAMILY_PREFIX);
+    public static final ImmutableBytesPtr LOCAL_INDEX_COLUMN_FAMILY_PREFIX_PTR = new ImmutableBytesPtr(
+        LOCAL_INDEX_COLUMN_FAMILY_PREFIX_BYTES);
+    
+    public static final String DEFAULT_LOCAL_INDEX_COLUMN_FAMILY = LOCAL_INDEX_COLUMN_FAMILY_PREFIX + DEFAULT_COLUMN_FAMILY;
+    public static final byte[] DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES = Bytes.toBytes(DEFAULT_LOCAL_INDEX_COLUMN_FAMILY);
+    public static final ImmutableBytesPtr DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES_PTR = new ImmutableBytesPtr(
+               DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES);
+
     public static final String ALL_FAMILY_PROPERTIES_KEY = "";
     public static final String SYSTEM_TABLE_PK_NAME = "pk";