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 611ba68..06c4c5c 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 @@ -81,7 +81,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"; 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(); 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 b450643..ac0f208 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 @@ -133,10 +133,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 + "',-32768,'" + 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 + "',-32768,'" + valuePrefix + "v2-1']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { @@ -156,7 +156,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 + ",-32768,'" + valuePrefix + "v2-1']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java index 3140077..2ce549e 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 @@ -132,7 +132,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_T [-32768,51]\n" + assertEquals("CLIENT PARALLEL "+ (saltBuckets == null ? 1 : saltBuckets) +"-WAY RANGE SCAN OVER T [-32768,51]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", queryPlan); @@ -168,7 +168,7 @@ public abstract class BaseViewIT extends BaseOwnClusterHBaseManagedTimeIT { assertFalse(rs.next()); rs = conn.createStatement().executeQuery("EXPLAIN " + query); if (localIndex) { - assertEquals("CLIENT PARALLEL "+ (saltBuckets == null ? 1 : saltBuckets) +"-WAY RANGE SCAN OVER _LOCAL_IDX_T [" + (Short.MIN_VALUE+1) + ",'foo']\n" + assertEquals("CLIENT PARALLEL "+ (saltBuckets == null ? 1 : saltBuckets) +"-WAY RANGE SCAN OVER T [" + (Short.MIN_VALUE+1) + ",'foo']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT",QueryUtil.getExplainPlan(rs)); } else { 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 745c730..1ea03de 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 @@ -160,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)"); } @@ -186,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.asList(5,10), indexName, false); + if(!local) { + assertIndexUsed(conn, deleteStmt, Arrays.asList(5,10), indexName, false); + } stmt = conn.prepareStatement(deleteStmt); stmt.setInt(1, 5); stmt.setInt(2, 10); @@ -204,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); + } stmt.execute(); if (!autoCommit) { conn.commit(); 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 88e03ca..4fd09c7 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 @@ -872,7 +872,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 +" [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT", /* @@ -886,7 +886,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 +" [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT", /* @@ -908,7 +908,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+" [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [\"I.0:NAME\"]\n" + "CLIENT MERGE SORT\n" + @@ -944,11 +944,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 + " [-32768,'T1'] - [-32768,'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 +" [-32768,'S1'] - [-32768,'S5']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT", /* @@ -959,10 +959,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 + " [-32768,'T1'] - [-32768,'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 +" [-32768,'S1'] - [-32768,'S5']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT", /* @@ -971,13 +971,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 + " [-32768]\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 + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT\n" + " DYNAMIC SERVER FILTER BY \"I.:item_id\" IN (\"O.item_id\")", @@ -989,7 +989,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 +" [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT\n" + " DYNAMIC SERVER FILTER BY \"I1.item_id\" IN (\"I2.:item_id\")", @@ -999,12 +999,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 +" [-32768]\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 +" [-32768]\n" + " CLIENT MERGE SORT\n" + " DYNAMIC SERVER FILTER BY \"I1.:item_id\" IN (\"I2.0:supplier_id\")", /* @@ -1017,11 +1017,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 + " [-32768]\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 + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT", /* @@ -1032,14 +1032,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 + " [-32768]\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+" [-32768]\n"+ " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT\n" + " DYNAMIC SERVER FILTER BY \"I.:item_id\" IN (\"O.item_id\")", @@ -1063,7 +1063,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 +" [-32768]\n" + " SERVER FILTER BY \"NAME\" != 'T3'\n" + " CLIENT MERGE SORT\n" + " PARALLEL LEFT-JOIN TABLE 0\n" + @@ -1080,7 +1080,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+" [-32768]\n"+ " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT", /* @@ -1096,7 +1096,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 + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT", /* @@ -1107,7 +1107,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 + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER SORTED BY [O.Q DESC NULLS LAST, I.IID]\n"+ "CLIENT MERGE SORT\n" + @@ -1123,7 +1123,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 + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER SORTED BY [O.Q DESC, I.IID]\n"+ "CLIENT MERGE SORT\n" + @@ -1157,7 +1157,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 + " [-32768]\n" + " SERVER FILTER BY \"NAME\" != 'T3'\n" + " CLIENT MERGE SORT\n" + " PARALLEL LEFT-JOIN TABLE 0\n" + @@ -1173,7 +1173,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 + " [-32768]\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" + @@ -1188,7 +1188,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 + " [-32768]\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" + @@ -1250,7 +1250,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 + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT\n" + "CLIENT 4 ROW LIMIT\n" + 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..ff71509 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 @@ -98,30 +98,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 + " [-32768,'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 + " [-32768,*] - [-32768,'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 + " [-32768,'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 + " [-32768,*] - [-32768,'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 + " [-32768,*] - [-32768,'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 + " [-32768,*] - [-32768,'T6']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT", }}); 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 9b85d8a..1e58f4a 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 @@ -187,13 +187,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 + " [-32768]\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 + " [-32768]\n" + " SERVER SORTED BY [\"I.:item_id\"]\n" + " CLIENT MERGE SORT\n" + " AND (SKIP MERGE)\n" + @@ -204,7 +204,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 + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER SORTED BY [\"I.:item_id\"]\n" + " CLIENT MERGE SORT\n" + @@ -215,12 +215,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 [-32768]\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 [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER SORTED BY [\"I2.:item_id\"]\n" + " CLIENT MERGE SORT\n" + 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 794c4f5..f1c54dc 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 @@ -245,10 +245,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 + " \\[-32768\\]\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 + " \\[-32768\\]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " CLIENT MERGE SORT\n" + " PARALLEL SEMI-JOIN TABLE 1 \\(SKIP MERGE\\)\n" + @@ -257,12 +257,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 + " [-32768]\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 + " [-32768]\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 +272,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 + " \\[-32768\\]\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 +281,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 + " \\[-32768\\]\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 +292,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 + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT\n" + " PARALLEL ANTI-JOIN TABLE 0 (SKIP MERGE)\n" + @@ -300,11 +300,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 + " \\[-32768\\]\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 + " \\[-32768\\]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"O.customer_id\"\\]\n" + " CLIENT MERGE SORT\n" + 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 f3b6bce..51da0b8 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 @@ -234,11 +234,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 + " [-32768]\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 + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER SORTED BY [\"S.:supplier_id\"]\n" + " CLIENT MERGE SORT\n" + @@ -254,7 +254,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 + " \\[-32768\\]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\".+.:item_id\", \".+.0:NAME\"\\]\n" + " CLIENT MERGE SORT\n" + @@ -264,7 +264,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 + " \\[-32768\\]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY \\[\".+.:item_id\", \".+.0:NAME\"\\]\n" + " CLIENT MERGE SORT\n" + @@ -276,12 +276,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 + " \\[-32768\\]\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 + " \\[-32768\\]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER AGGREGATE INTO DISTINCT ROWS BY \\[\"O.customer_id\"\\]\n" + " CLIENT MERGE SORT\n" + 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..b693a25 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',-32768,'f']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT",QueryUtil.getExplainPlan(rs)); } else { 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 4d1c5e4..2f70986 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 @@ -779,7 +779,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 [-32768,'kcoj']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" +"CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); rs = stmt.executeQuery(query); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java index ea12245..bb671c4 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java @@ -155,7 +155,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { String query = "SELECT d.char_col1, int_col1 from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " as d"; ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query); if (localIndex) { - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_FULL_NAME)+" [-32768]\n" + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME +" [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { @@ -202,7 +202,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { String query = "SELECT int_pk from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME ; ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query); if (localIndex) { - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME +" [-32768]\n" + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME +" [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { @@ -222,7 +222,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT date_col from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " order by date_col" ; rs = conn.createStatement().executeQuery("EXPLAIN " + query); if (localIndex) { - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " [-32768]\n" + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { @@ -268,7 +268,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { String query = "SELECT char_col1, int_col1, long_col2 from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME; ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query); if (localIndex) { - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME +" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME +" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME, QueryUtil.getExplainPlan(rs)); } @@ -335,7 +335,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { if(localIndex) { query = "SELECT b.* from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " where int_col1 = 4"; rs = conn.createStatement().executeQuery("EXPLAIN " + query); - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME +" [-32768]\n" + + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME +" [-32768]\n" + " SERVER FILTER BY TO_INTEGER(\"INT_COL1\") = 4\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); rs = conn.createStatement().executeQuery(query); assertTrue(rs.next()); @@ -387,7 +387,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME; rs = conn.createStatement().executeQuery("EXPLAIN " + query); if(localIndex){ - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME, QueryUtil.getExplainPlan(rs)); } @@ -412,7 +412,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT v1 as foo FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " WHERE v2 = '1' ORDER BY foo"; rs = conn.createStatement().executeQuery("EXPLAIN " + query); if(localIndex){ - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" +TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " [-32768,~'1']\n" + + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " [-32768,~'1']\n" + " SERVER SORTED BY [\"V1\"]\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { @@ -469,7 +469,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT a.* FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME; rs = conn.createStatement().executeQuery("EXPLAIN " + query); if(localIndex) { - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME, QueryUtil.getExplainPlan(rs)); } @@ -541,7 +541,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME; rs = conn.createStatement().executeQuery("EXPLAIN " + query); if(localIndex) { - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME, QueryUtil.getExplainPlan(rs)); } @@ -562,7 +562,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME; rs = conn.createStatement().executeQuery("EXPLAIN " + query); if(localIndex) { - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME, QueryUtil.getExplainPlan(rs)); } @@ -583,7 +583,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME; rs = conn.createStatement().executeQuery("EXPLAIN " + query); if(localIndex) { - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME, QueryUtil.getExplainPlan(rs)); } @@ -653,7 +653,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME; rs = conn.createStatement().executeQuery("EXPLAIN " + query); if (localIndex) { - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\n" + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { @@ -784,7 +784,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME; rs = conn.createStatement().executeQuery("EXPLAIN " + query); if(localIndex) { - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\n" + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -930,7 +930,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME; rs = conn.createStatement().executeQuery("EXPLAIN " + query); if (localIndex) { - assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\n" + assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -993,7 +993,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT * FROM cs WHERE \"v2\" = '1'"; rs = conn.createStatement().executeQuery("EXPLAIN " + query); if(localIndex){ - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_CS [-32768,'1']\n" + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER CS [-32768,'1']\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER ICS ['1']", QueryUtil.getExplainPlan(rs)); @@ -1012,7 +1012,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { query = "SELECT \"V1\", \"V1\" as foo1, \"v2\" as foo, \"v2\" as \"Foo1\", \"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 [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER ICS", QueryUtil.getExplainPlan(rs)); @@ -1092,7 +1092,7 @@ public abstract class BaseMutableIndexIT extends BaseHBaseManagedTimeIT { String query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME ; ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query); if(localIndex) { - assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_" + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); + assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME+" [-32768]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); } else { assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME, QueryUtil.getExplainPlan(rs)); } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java index 9eb9a57..8675e1d 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java @@ -149,7 +149,7 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT { ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query); if(localIndex) { assertEquals( - "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_INDEX_TEST.INDEX_DATA_TABLE [-32768]\n" + + "CLIENT PARALLEL 1-WAY RANGE SCAN OVER INDEX_TEST.INDEX_DATA_TABLE [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java index 5e01510..b805c4a 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,17 +24,18 @@ 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; import java.sql.SQLException; import java.sql.Statement; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.concurrent.CountDownLatch; +import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaTableAccessor; @@ -44,12 +45,7 @@ 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; @@ -60,6 +56,7 @@ 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; @@ -80,9 +77,6 @@ import com.google.common.collect.Maps; public class LocalIndexIT extends BaseHBaseManagedTimeIT { - private static CountDownLatch latch1 = new CountDownLatch(1); - private static CountDownLatch latch2 = new CountDownLatch(1); - @BeforeClass @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class) public static void doSetup() throws Exception { @@ -158,13 +152,8 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { conn1.createStatement().execute("CREATE LOCAL INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_NAME + "(v1)"); conn2.createStatement().executeQuery("SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME).next(); HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin(); - HTableDescriptor htd = admin.getTableDescriptor(TableName.valueOf(MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME))); + HTableDescriptor htd = admin.getTableDescriptor(TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME)); assertEquals(IndexRegionSplitPolicy.class.getName(), htd.getValue(HTableDescriptor.SPLIT_POLICY)); - try (HTable userTable = new HTable(admin.getConfiguration(),TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME))) { - try (HTable indexTable = new HTable(admin.getConfiguration(),TableName.valueOf(MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME)))) { - assertArrayEquals("Both user table and index table should have same split keys.", userTable.getStartKeys(), indexTable.getStartKeys()); - } - } } @Test @@ -174,12 +163,8 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { Connection conn2 = DriverManager.getConnection(getUrl()); conn1.createStatement().execute("CREATE LOCAL INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_NAME + "(v1)"); conn2.createStatement().executeQuery("SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME).next(); - HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin(); - assertTrue("Local index table should be present.", admin.tableExists(TableName.valueOf(MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME)))); conn1.createStatement().execute("DROP TABLE "+ TestUtil.DEFAULT_DATA_TABLE_NAME); - admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin(); - assertFalse("Local index table should be deleted.", admin.tableExists(TableName.valueOf(MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME)))); - ResultSet rs = conn2.createStatement().executeQuery("SELECT " + ResultSet rs = conn1.createStatement().executeQuery("SELECT " + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + "," + PhoenixDatabaseMetaData.SEQUENCE_NAME + " FROM " + PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_ESCAPED); @@ -200,14 +185,23 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { assertTrue(rs.next()); assertEquals(4, rs.getInt(1)); HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin(); - HTable indexTable = new HTable(admin.getConfiguration() ,TableName.valueOf(MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME))); + HTable indexTable = new HTable(admin.getConfiguration() ,TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME)); Pair startEndKeys = indexTable.getStartEndKeys(); + HTableDescriptor tableDescriptor = indexTable.getTableDescriptor(); + byte[] cfBytes = null; + for(HColumnDescriptor cf: tableDescriptor.getFamilies()) { + if(cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)){ + cfBytes = cf.getName(); + } + } + byte[][] startKeys = startEndKeys.getFirst(); byte[][] endKeys = startEndKeys.getSecond(); for (int i = 0; i < startKeys.length; i++) { Scan s = new Scan(); s.setStartRow(startKeys[i]); s.setStopRow(endKeys[i]); + s.addFamily(cfBytes); ResultScanner scanner = indexTable.getScanner(s); int count = 0; for(Result r:scanner){ @@ -233,14 +227,22 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { assertTrue(rs.next()); assertEquals(4, rs.getInt(1)); HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin(); - HTable indexTable = new HTable(admin.getConfiguration() ,TableName.valueOf(MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME))); + HTable indexTable = new HTable(admin.getConfiguration() ,TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME)); Pair startEndKeys = indexTable.getStartEndKeys(); byte[][] startKeys = startEndKeys.getFirst(); byte[][] endKeys = startEndKeys.getSecond(); + HTableDescriptor tableDescriptor = indexTable.getTableDescriptor(); + byte[] cfBytes = null; + for(HColumnDescriptor cf: tableDescriptor.getFamilies()) { + if(cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)){ + cfBytes = cf.getName(); + } + } for (int i = 0; i < startKeys.length; i++) { Scan s = new Scan(); s.setStartRow(startKeys[i]); s.setStopRow(endKeys[i]); + s.addFamily(cfBytes); ResultScanner scanner = indexTable.getScanner(s); int count = 0; for(Result r:scanner){ @@ -257,6 +259,8 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { createBaseTable(TestUtil.DEFAULT_DATA_TABLE_NAME, null, "('e','i','o')"); Connection conn1 = DriverManager.getConnection(getUrl()); try{ + conn1.createStatement().execute("CREATE LOCAL INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_NAME + "(v1)"); + conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('a',1,2,5,'y')"); conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('b',1,2,4,'z')"); conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('f',1,2,3,'a')"); @@ -264,8 +268,8 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('j',2,4,2,'a')"); conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('q',3,1,1,'c')"); conn1.commit(); - conn1.createStatement().execute("CREATE LOCAL INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_NAME + "(v1)"); - + + ResultSet rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + TestUtil.DEFAULT_INDEX_TABLE_NAME); assertTrue(rs.next()); @@ -277,7 +281,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { assertEquals( "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER " - + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME) + " [-32768,'a'] - [-32768,'b']\n" + + TestUtil.DEFAULT_DATA_TABLE_NAME + " [-32768,'a'] - [-32768,'b']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -301,7 +305,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { assertEquals( "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER " - + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME) + " [-32768,'a']\n" + + TestUtil.DEFAULT_DATA_TABLE_NAME + " [-32768,'a']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -320,7 +324,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { rs = conn1.createStatement().executeQuery("EXPLAIN "+ query); assertEquals( - "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER " + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME) + " [-32768,*] - [-32768,'z']\n" + "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_NAME + " [-32768,*] - [-32768,'z']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER SORTED BY [\"K3\"]\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -345,7 +349,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { assertEquals( "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER " - + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME)+" [-32768]\n" + + TestUtil.DEFAULT_DATA_TABLE_NAME+" [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -409,7 +413,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { assertEquals( "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER " - + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME) + " [-32768,'a']\n" + + TestUtil.DEFAULT_DATA_TABLE_NAME + " [-32768,'a']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -432,7 +436,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { assertEquals( "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER " - + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME)+" [-32768,*] - [-32768,'z']\n" + + TestUtil.DEFAULT_DATA_TABLE_NAME +" [-32768,*] - [-32768,'z']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -468,7 +472,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { assertEquals( "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER " - + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME)+" [-32768,*] - [-32768,'z']\n" + + TestUtil.DEFAULT_DATA_TABLE_NAME+" [-32768,*] - [-32768,'z']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER AGGREGATE INTO DISTINCT ROWS BY [\"V1\", \"T_ID\", \"K3\"]\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -496,7 +500,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { rs = conn1.createStatement().executeQuery("EXPLAIN " + query); assertEquals( "CLIENT PARALLEL " + numRegions + "-WAY RANGE SCAN OVER " - + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME)+" [-32768,*] - [-32768,'z']\n" + + TestUtil.DEFAULT_DATA_TABLE_NAME+" [-32768,*] - [-32768,'z']\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + " SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [\"V1\"]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -551,8 +555,8 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { conn1.createStatement().execute("CREATE LOCAL INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_NAME + "(v1)"); conn1.createStatement().execute("DROP INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_NAME); HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin(); - HTable indexTable = new HTable(admin.getConfiguration() ,TableName.valueOf(MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME))); - Pair startEndKeys = indexTable.getStartEndKeys(); + HTable table = new HTable(admin.getConfiguration() ,TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME)); + Pair startEndKeys = table.getStartEndKeys(); byte[][] startKeys = startEndKeys.getFirst(); byte[][] endKeys = startEndKeys.getSecond(); // No entry should be present in local index table after drop index. @@ -560,7 +564,13 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { Scan s = new Scan(); s.setStartRow(startKeys[i]); s.setStopRow(endKeys[i]); - ResultScanner scanner = indexTable.getScanner(s); + Collection 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++; @@ -568,7 +578,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { scanner.close(); assertEquals(0, count); } - indexTable.close(); + table.close(); } finally { conn1.close(); } @@ -666,12 +676,12 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { createBaseTable(TestUtil.DEFAULT_DATA_TABLE_NAME, null, "('e','i','o')"); Connection conn1 = DriverManager.getConnection(getUrl()); try{ + conn1.createStatement().execute("CREATE LOCAL INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_NAME + "(v1) include (k3)"); conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('b',1,2,4,'z')"); conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('f',1,2,3,'a')"); conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('j',2,4,2,'a')"); conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('q',3,1,1,'c')"); conn1.commit(); - conn1.createStatement().execute("CREATE LOCAL INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_NAME + "(v1) include (k3)"); ResultSet rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + TestUtil.DEFAULT_INDEX_TABLE_NAME); assertTrue(rs.next()); @@ -680,7 +690,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 { @@ -737,18 +747,6 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { admin.getConnection(), TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME), false); } assertEquals(4+i, regionsOfUserTable.size()); - TableName indexTable = - TableName.valueOf(MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME)); - List 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 query = "SELECT t_id,k1,v1 FROM " + TestUtil.DEFAULT_DATA_TABLE_NAME; rs = conn1.createStatement().executeQuery(query); Thread.sleep(1000); @@ -761,7 +759,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { rs = conn1.createStatement().executeQuery("EXPLAIN " + query); assertEquals( "CLIENT PARALLEL " + (4 + i) + "-WAY RANGE SCAN OVER " - + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME) + " [-32768]\n" + + TestUtil.DEFAULT_DATA_TABLE_NAME + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -771,7 +769,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { "CLIENT PARALLEL " + ((strings[3 * i].compareTo("j") < 0) ? (4 + i) : (4 + i - 1)) + "-WAY RANGE SCAN OVER " - + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME) + " [-32767]\n" + + TestUtil.DEFAULT_DATA_TABLE_NAME + " [-32767]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); rs = conn1.createStatement().executeQuery(query); @@ -833,7 +831,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { @Test public void testLocalIndexScanAfterRegionsMerge() throws Exception { createBaseTable(TestUtil.DEFAULT_DATA_TABLE_NAME, null, "('e','j','o')"); - Connection conn1 = DriverManager.getConnection(getUrl()); + PhoenixConnection conn1 = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class); 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++) { @@ -848,7 +846,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { ResultSet rs = conn1.createStatement().executeQuery("SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_NAME); assertTrue(rs.next()); - HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin(); + HBaseAdmin admin = conn1.getQueryServices().getAdmin(); List regionsOfUserTable = MetaTableAccessor.getTableRegions(getUtility().getZooKeeperWatcher(), admin.getConnection(), TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME), false); @@ -857,29 +855,15 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { regionsOfUserTable = MetaTableAccessor.getTableRegions(getUtility().getZooKeeperWatcher(), admin.getConnection(), TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME), false); - while (regionsOfUserTable.size() != 3) { Thread.sleep(100); regionsOfUserTable = MetaTableAccessor.getTableRegions(getUtility().getZooKeeperWatcher(), admin.getConnection(), TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME), false); } assertEquals(3, regionsOfUserTable.size()); - TableName indexTable = - TableName.valueOf(MetaDataUtil - .getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME)); - List 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 " + TestUtil.DEFAULT_DATA_TABLE_NAME; + conn1.getQueryServices().clearTableRegionCache(Bytes.toBytes(TestUtil.DEFAULT_DATA_TABLE_NAME)); + String query = "SELECT t_id,k1,v1 FROM " + TestUtil.DEFAULT_DATA_TABLE_NAME; rs = conn1.createStatement().executeQuery(query); - Thread.sleep(1000); for (int j = 0; j < 26; j++) { assertTrue(rs.next()); assertEquals(strings[25 - j], rs.getString("t_id")); @@ -889,7 +873,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { rs = conn1.createStatement().executeQuery("EXPLAIN " + query); assertEquals( "CLIENT PARALLEL " + 3 + "-WAY RANGE SCAN OVER " - + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME) + + TestUtil.DEFAULT_DATA_TABLE_NAME + " [-32768]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -897,7 +881,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { rs = conn1.createStatement().executeQuery("EXPLAIN " + query); assertEquals( "CLIENT PARALLEL " + 3 + "-WAY RANGE SCAN OVER " - + MetaDataUtil.getLocalIndexTableName(TestUtil.DEFAULT_DATA_TABLE_NAME) + +TestUtil.DEFAULT_DATA_TABLE_NAME + " [-32767]\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs)); @@ -913,105 +897,4 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT { conn1.close(); } } - - @Test - public void testLocalIndexStateWhenSplittingInProgress() throws Exception { - createBaseTable(TestUtil.DEFAULT_DATA_TABLE_NAME+"2", null, "('e','j','o')"); - Connection conn1 = DriverManager.getConnection(getUrl()); - 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 " + TestUtil.DEFAULT_DATA_TABLE_NAME+"2" + " values('"+strings[i]+"'," + i + "," - + (i + 1) + "," + (i + 2) + ",'" + strings[25 - i] + "')"); - } - conn1.commit(); - conn1.createStatement().execute("CREATE LOCAL INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_NAME+"2" + "(v1)"); - conn1.createStatement().execute("CREATE LOCAL INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + "_2 ON " + TestUtil.DEFAULT_DATA_TABLE_NAME+"2" + "(k3)"); - - ResultSet rs = conn1.createStatement().executeQuery("SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_NAME+"2"); - assertTrue(rs.next()); - HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin(); - HTableDescriptor tableDesc = admin.getTableDescriptor(TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME+"2")); - 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(TestUtil.DEFAULT_DATA_TABLE_NAME+"2")); - 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(Bytes.toBytes(TestUtil.DEFAULT_DATA_TABLE_NAME+"2"), ByteUtil.concat(Bytes.toBytes(strings[3]))); - List regionsOfUserTable = - admin.getTableRegions(TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME+"2")); - - while (regionsOfUserTable.size() != 5) { - Thread.sleep(100); - regionsOfUserTable = admin.getTableRegions(TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME+"2")); - } - assertEquals(5, regionsOfUserTable.size()); - - List regionsOfIndexTable = admin.getTableRegions(indexTable); - - while (regionsOfIndexTable.size() != 5) { - Thread.sleep(100); - regionsOfIndexTable = admin.getTableRegions(indexTable); - } - - assertEquals(5, regionsOfIndexTable.size()); - latch1.await(); - // Verify the metadata for index is correct. - rs = conn1.getMetaData().getTables(null, StringUtil.escapeLike(TestUtil.DEFAULT_SCHEMA_NAME), TestUtil.DEFAULT_INDEX_TABLE_NAME, - new String[] { PTableType.INDEX.toString() }); - assertTrue(rs.next()); - assertEquals(TestUtil.DEFAULT_INDEX_TABLE_NAME, rs.getString(3)); - assertEquals(PIndexState.INACTIVE.toString(), rs.getString("INDEX_STATE")); - assertFalse(rs.next()); - rs = conn1.getMetaData().getTables(null, StringUtil.escapeLike(TestUtil.DEFAULT_SCHEMA_NAME), TestUtil.DEFAULT_INDEX_TABLE_NAME+"_2", - new String[] { PTableType.INDEX.toString() }); - assertTrue(rs.next()); - assertEquals(TestUtil.DEFAULT_INDEX_TABLE_NAME+"_2", rs.getString(3)); - assertEquals(PIndexState.INACTIVE.toString(), rs.getString("INDEX_STATE")); - assertFalse(rs.next()); - - String query = "SELECT t_id,k1,v1 FROM " + TestUtil.DEFAULT_DATA_TABLE_NAME+"2"; - rs = conn1.createStatement().executeQuery("EXPLAIN " + query); - assertEquals("CLIENT PARALLEL " + 1 + "-WAY FULL SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_NAME+"2", - QueryUtil.getExplainPlan(rs)); - latch2.countDown(); - } finally { - conn1.close(); - latch1.countDown(); - latch2.countDown(); - } - } - - public static class MockedIndexHalfStoreFileReaderGenerator extends IndexHalfStoreFileReaderGenerator { - @Override - public void postCompact(ObserverContext e, Store store, - StoreFile resultFile) throws IOException { - try { - latch2.await(); - } catch (InterruptedException e1) { - } - super.postCompact(e, store, resultFile); - } - } - - public static class MockedLocalIndexSplitter extends LocalIndexSplitter { - @Override - public void preSplitAfterPONR(ObserverContext ctx) - throws IOException { - super.preSplitAfterPONR(ctx); - latch1.countDown(); - } - } } 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 364b358..a63e043 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 @@ -44,7 +44,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseCluster; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.ServerName; @@ -52,12 +51,8 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.master.LoadBalancer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; -import org.apache.phoenix.hbase.index.balancer.IndexLoadBalancer; -import org.apache.phoenix.hbase.index.master.IndexMasterObserver; import org.apache.phoenix.jdbc.PhoenixTestDriver; import org.apache.phoenix.query.BaseTest; import org.apache.phoenix.query.QueryServices; @@ -105,9 +100,6 @@ public class MutableIndexFailureIT extends BaseTest { conf.setInt("hbase.client.pause", 5000); conf.setInt("hbase.balancer.period", Integer.MAX_VALUE); conf.setLong(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB, 0); - conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, IndexMasterObserver.class.getName()); - conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, IndexLoadBalancer.class, - LoadBalancer.class); util = new HBaseTestingUtility(conf); util.startMiniCluster(NUM_SLAVES); String clientPort = util.getConfiguration().get(QueryServices.ZOOKEEPER_PORT_ATTRIB); 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 449dccf..0000000 --- a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancerIT.java +++ /dev/null @@ -1,491 +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 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.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category(NeedsOwnMiniClusterTest.class) -public class IndexLoadBalancerIT { - - private static HBaseTestingUtility UTIL = new HBaseTestingUtility(); - - private static HBaseAdmin admin = null; - - @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(); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - try { - if (admin != null) { - admin.disableTables(".*"); - admin.deleteTables(".*"); - admin.close(); - } - } finally { - UTIL.shutdownMiniCluster(); - } - } - - @Test(timeout = 180000) - 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(timeout = 180000) - 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 regionsOfUserTable = - master.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName); - - while (regionsOfUserTable.size() != 22) { - Thread.sleep(100); - regionsOfUserTable = - master.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName); - } - - List 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(timeout = 180000) - 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 regionsOfUserTable = regionStates.getRegionsOfTable(tableName); - Pair regionsToMerge = new Pair(); - 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 regionsOfIndexTable = regionStates.getRegionsOfTable(indexTableName); - Pair indexRegionsToMerge = new Pair(); - 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(timeout = 180000) - 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(timeout = 180000) - 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(timeout = 180000) - public void testBalanceByTable() throws Exception { - ZooKeeperWatcher zkw = UTIL.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(timeout = 180000) - public void testRoundRobinAssignmentAfterRegionServerDown() throws Exception { - ZooKeeperWatcher zkw = UTIL.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(timeout = 180000) - public void testRetainAssignmentDuringMasterStartUp() throws Exception { - ZooKeeperWatcher zkw = UTIL.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); - - } - - @Test(timeout = 300000) - 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> getStartKeysAndLocations(HMaster master, String tableName) - throws IOException, InterruptedException { - - List> tableRegionsAndLocations = - MetaTableAccessor.getTableRegionsAndLocations(master.getZooKeeper(), master.getConnection(), - TableName.valueOf(tableName)); - List> startKeyAndLocationPairs = - new ArrayList>(tableRegionsAndLocations.size()); - Pair startKeyAndLocation = null; - for (Pair regionAndLocation : tableRegionsAndLocations) { - startKeyAndLocation = - new Pair(regionAndLocation.getFirst().getStartKey(), - regionAndLocation.getSecond()); - startKeyAndLocationPairs.add(startKeyAndLocation); - } - return startKeyAndLocationPairs; - - } - - public boolean checkForColocation(HMaster master, String tableName, String indexTableName) - throws IOException, InterruptedException { - List> uTableStartKeysAndLocations = - getStartKeysAndLocations(master, tableName); - List> iTableStartKeysAndLocations = - getStartKeysAndLocations(master, indexTableName); - - boolean regionsColocated = true; - if (uTableStartKeysAndLocations.size() != iTableStartKeysAndLocations.size()) { - regionsColocated = false; - } else { - for (int i = 0; i < uTableStartKeysAndLocations.size(); i++) { - Pair uStartKeyAndLocation = uTableStartKeysAndLocations.get(i); - Pair iStartKeyAndLocation = iTableStartKeysAndLocations.get(i); - - if (Bytes.compareTo(uStartKeyAndLocation.getFirst(), iStartKeyAndLocation - .getFirst()) == 0) { - if (uStartKeyAndLocation.getSecond().equals(iStartKeyAndLocation.getSecond())) { - continue; - } - } - regionsColocated = false; - } - } - return regionsColocated; - } -} diff --git a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java index e2ebad4..4eb9325 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java @@ -36,7 +36,13 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.client.HTableInterface; +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.util.Bytes; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDriver; import org.apache.phoenix.util.DateUtil; import org.apache.phoenix.util.PhoenixRuntime; @@ -200,7 +206,7 @@ public class CsvBulkLoadToolIT { Statement stmt = conn.createStatement(); stmt.execute("CREATE TABLE TABLE6 (ID INTEGER NOT NULL PRIMARY KEY, " + - "FIRST_NAME VARCHAR, LAST_NAME VARCHAR)"); + "FIRST_NAME VARCHAR, LAST_NAME VARCHAR) SPLIT ON (1,2)"); String ddl = "CREATE LOCAL INDEX TABLE6_IDX ON TABLE6 " + " (FIRST_NAME ASC)"; stmt.execute(ddl); @@ -216,16 +222,18 @@ public class CsvBulkLoadToolIT { CsvBulkLoadTool csvBulkLoadTool = new CsvBulkLoadTool(); csvBulkLoadTool.setConf(hbaseTestUtil.getConfiguration()); - try { - csvBulkLoadTool.run(new String[] { - "--input", "/tmp/input3.csv", - "--table", "table6", - "--zookeeper", zkQuorum}); - fail("Csv bulk load currently has issues with local indexes."); - } catch( UnsupportedOperationException ise) { - assertEquals("Local indexes not supported by CSV Bulk Loader",ise.getMessage()); - } - + csvBulkLoadTool.run(new String[] { + "--input", "/tmp/input3.csv", + "--table", "table6", + "--zookeeper", zkQuorum}); + ResultSet rs = stmt.executeQuery("SELECT * FROM TABLE6"); + assertTrue(rs.next()); + rs = stmt.executeQuery("SELECT FIRST_NAME FROM TABLE6 where FIRST_NAME='FirstName 1'"); + assertTrue(rs.next()); + assertEquals("FirstName 1", rs.getString(1)); + + rs.close(); + stmt.close(); } @Test @@ -233,7 +241,7 @@ public class CsvBulkLoadToolIT { testImportOneIndexTable("TABLE4", false); } - //@Test + @Test public void testImportOneLocalIndexTable() throws Exception { testImportOneIndexTable("TABLE5", true); } @@ -243,7 +251,7 @@ public class CsvBulkLoadToolIT { String indexTableName = String.format("%s_IDX", tableName); Statement stmt = conn.createStatement(); stmt.execute("CREATE TABLE " + tableName + "(ID INTEGER NOT NULL PRIMARY KEY, " - + "FIRST_NAME VARCHAR, LAST_NAME VARCHAR)"); + + "FIRST_NAME VARCHAR, LAST_NAME VARCHAR) SPLIT ON(1,2)"); String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexTableName + " ON " + tableName + "(FIRST_NAME ASC)"; @@ -264,9 +272,11 @@ public class CsvBulkLoadToolIT { "--index-table", indexTableName, "--zookeeper", zkQuorum }); assertEquals(0, exitCode); - - ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName); - assertFalse(rs.next()); + ResultSet rs = null; + if(!localIndex) { + rs = stmt.executeQuery("SELECT * FROM " + tableName); + assertFalse(rs.next()); + } rs = stmt.executeQuery("SELECT FIRST_NAME FROM " + tableName + " where FIRST_NAME='FirstName 1'"); assertTrue(rs.next()); assertEquals("FirstName 1", rs.getString(1)); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/IndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/IndexToolIT.java index 30cae36..ab62278 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/IndexToolIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/IndexToolIT.java @@ -276,7 +276,7 @@ public class IndexToolIT { String expectedExplainPlan = ""; if(isLocal) { - final String localIndexName = MetaDataUtil.getLocalIndexTableName(SchemaUtil.getTableName(schemaName, dataTable)); + final String localIndexName = SchemaUtil.getTableName(schemaName, dataTable); expectedExplainPlan = String.format("CLIENT 1-CHUNK PARALLEL 1-WAY RANGE SCAN OVER %s [-32768]" + "\n SERVER FILTER BY FIRST KEY ONLY", localIndexName); } else { 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 94d5912..6595bda 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.phoenix.index.IndexMaintainer; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.parse.AlterIndexStatement; import org.apache.phoenix.parse.ParseNodeFactory; +import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.schema.MetaDataClient; import org.apache.phoenix.schema.PColumn; import org.apache.phoenix.schema.PIndexState; @@ -80,6 +81,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())) { @@ -110,7 +114,9 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver { MetaTableAccessor.getRegionsFromMergeQualifier(ctx.getEnvironment() .getRegionServerServices().getConnection(), region.getRegionInfo().getRegionName()); - if (mergeRegions == null || mergeRegions.getFirst() == null) return reader; + if (mergeRegions == null || mergeRegions.getFirst() == null) { + return reader; + } byte[] splitRow = CellUtil.cloneRow(KeyValue.createKeyValueFromKey(r.getSplitKey())); // We need not change any thing in first region data because first region start key @@ -136,8 +142,7 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver { try { conn = QueryUtil.getConnection(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 indexes = dataTable.getIndexes(); Map indexMaintainers = new HashMap(); @@ -171,59 +176,6 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver { return reader; } - @Override - public InternalScanner preCompactScannerOpen(ObserverContext c, - Store store, List scanners, ScanType scanType, - long earliestPutTs, InternalScanner s, CompactionRequest request) throws IOException { - InternalScanner internalScanner = super.preCompactScannerOpen(c, store, scanners, scanType, earliestPutTs, s, request); - Collection files = request.getFiles(); - storeFilesCount = 0; - compactedFilesCount = 0; - for(StoreFile file:files) { - if(!file.isReference()) { - return internalScanner; - } - } - storeFilesCount = files.size(); - return internalScanner; - } - - @Override - public void postCompact(ObserverContext e, Store store, - StoreFile resultFile) throws IOException { - super.postCompact(e, store, resultFile); - if(storeFilesCount > 0) compactedFilesCount++; - if(compactedFilesCount == storeFilesCount) { - PhoenixConnection conn = null; - try { - conn = QueryUtil.getConnection(e.getEnvironment().getConfiguration()).unwrap( - PhoenixConnection.class); - MetaDataClient client = new MetaDataClient(conn); - String userTableName = MetaDataUtil.getUserTableName(e.getEnvironment().getRegion().getTableDesc().getNameAsString()); - PTable dataTable = PhoenixRuntime.getTable(conn, userTableName); - List indexes = dataTable.getIndexes(); - for (PTable index : indexes) { - if (index.getIndexType() == IndexType.LOCAL) { - AlterIndexStatement indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null, - org.apache.phoenix.parse.TableName.create(index.getSchemaName().getString(), index.getTableName().getString())), - dataTable.getTableName().getString(), false, PIndexState.ACTIVE); - client.alterIndex(indexStatement); - } - } - conn.commit(); - } catch (ClassNotFoundException ex) { - } catch (SQLException ex) { - } finally { - if (conn != null) { - try { - conn.close(); - } catch (SQLException ex) { - } - } - } - } - } - private byte[][] getViewConstants(PTable dataTable) { int dataPosOffset = (dataTable.getBucketNum() != null ? 1 : 0) + (dataTable.isMultiTenant() ? 1 : 0); byte[][] viewConstants = null; 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 71bc520..0000000 --- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java +++ /dev/null @@ -1,984 +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 journal = new ArrayList(); - - /** - * 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 true if the region is splittable else - * false 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 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 daughterRegions = stepsBeforePONR(server, services, testing); - - List metaEntries = new ArrayList(); - 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 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> 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(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 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 execute(final Server server, - final RegionServerServices services) - throws IOException { - PairOfSameType regions = createDaughters(server, services); - if (this.parent.getCoprocessorHost() != null) { - this.parent.getCoprocessorHost().preSplitAfterPONR(); - } - return stepsAfterPONR(server, services, regions); - } - - @Override - public PairOfSameType stepsAfterPONR(final Server server, - final RegionServerServices services, PairOfSameType 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 metaEntries, int regionReplication) throws IOException { - List 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> 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> futures = new ArrayList>(nbFiles); - - // Split each store file. - for (Map.Entry> 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) { - throw (InterruptedIOException)new InterruptedIOException().initCause(e); - } - - // Look for any exception - for (Future future: futures) { - try { - future.get(); - } catch (InterruptedException e) { - 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 { - 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 f -- 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 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. - * - *

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). - * - *

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. - * - *

This method can fail and return false for three different reasons: - *

  • Node for this region does not exist
  • - *
  • Node for this region is not in the begin state
  • - *
  • 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)
  • - *
- * - *

Does not set any watches. - * - *

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 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 ctx, - Region regionA, Region regionB, List 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 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 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"); - } - } -} 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 deleted file mode 100644 index 7882e25..0000000 --- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexSplitter.java +++ /dev/null @@ -1,190 +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 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.util.PairOfSameType; -import org.apache.phoenix.hbase.index.util.VersionUtil; -import org.apache.phoenix.jdbc.PhoenixConnection; -import org.apache.phoenix.parse.AlterIndexStatement; -import org.apache.phoenix.parse.ParseNodeFactory; -import org.apache.phoenix.schema.MetaDataClient; -import org.apache.phoenix.schema.PIndexState; -import org.apache.phoenix.schema.PTable; -import org.apache.phoenix.schema.PTable.IndexType; -import org.apache.phoenix.schema.types.PBoolean; -import org.apache.phoenix.util.IndexUtil; -import org.apache.phoenix.util.MetaDataUtil; -import org.apache.phoenix.util.PhoenixRuntime; -import org.apache.phoenix.util.QueryUtil; -import org.apache.phoenix.util.SchemaUtil; - -import java.io.IOException; -import java.sql.SQLException; -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 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 ctx, - byte[] splitKey, List metaEntries) throws IOException { - RegionCoprocessorEnvironment environment = ctx.getEnvironment(); - HTableDescriptor tableDesc = ctx.getEnvironment().getRegion().getTableDesc(); - if (SchemaUtil.isSystemTable(tableDesc.getName())) { - return; - } - 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); - daughterRegions = st.stepsBeforePONR(rss, rss, false); - 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 ctx) - throws IOException { - if (st == null || daughterRegions == null) return; - RegionCoprocessorEnvironment environment = ctx.getEnvironment(); - PhoenixConnection conn = null; - try { - conn = QueryUtil.getConnection(ctx.getEnvironment().getConfiguration()).unwrap( - PhoenixConnection.class); - MetaDataClient client = new MetaDataClient(conn); - String userTableName = ctx.getEnvironment().getRegion().getTableDesc().getNameAsString(); - PTable dataTable = PhoenixRuntime.getTable(conn, userTableName); - List indexes = dataTable.getIndexes(); - for (PTable index : indexes) { - if (index.getIndexType() == IndexType.LOCAL) { - AlterIndexStatement indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null, - org.apache.phoenix.parse.TableName.create(index.getSchemaName().getString(), index.getTableName().getString())), - dataTable.getTableName().getString(), false, PIndexState.INACTIVE); - client.alterIndex(indexStatement); - } - } - conn.commit(); - } catch (ClassNotFoundException ex) { - } catch (SQLException ex) { - } finally { - if (conn != null) { - try { - conn.close(); - } catch (SQLException ex) { - } - } - } - - HRegionServer rs = (HRegionServer) environment.getRegionServerServices(); - st.stepsAfterPONR(rs, rs, daughterRegions); - } - - @Override - public void preRollBackSplit(ObserverContext 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"); - } - } - -} 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 ad02961..f34bf62 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 @@ -300,7 +300,7 @@ public class ProjectionCompiler { throw e; } } - if (resolveColumn) { + if (resolveColumn && !(ref instanceof LocalIndexDataColumnRef)) { ref = context.getResolver().resolveColumn(index.getTableName().getString(), indexColumn.getFamilyName() == null ? null : indexColumn.getFamilyName().getString(), indexColName); } Expression expression = ref.newColumnExpression(); 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 56087c0..46ed776 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 @@ -20,6 +20,7 @@ package org.apache.phoenix.compile; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.Lists.newArrayListWithCapacity; +import java.io.IOException; import java.sql.Date; import java.sql.ParameterMetaData; import java.sql.ResultSet; @@ -34,6 +35,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; @@ -52,6 +58,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; @@ -82,11 +89,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; @@ -112,7 +122,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 mutation, PhoenixStatement statement, boolean useServerTimestamp) { + private static void setValues(byte[][] values, int[] pkSlotIndex, int[] columnIndexes, PTable table, Map mutation, PhoenixStatement statement, boolean useServerTimestamp) throws SQLException { Map 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 @@ -142,7 +152,34 @@ public class UpsertCompiler { } } ImmutableBytesPtr ptr = new ImmutableBytesPtr(); - table.newKey(ptr, pkValues); + if(table.getIndexType()==IndexType.LOCAL) { + PTable parentTable = statement.getConnection().getMetaDataCache().getTable(new PTableKey(statement.getConnection().getTenantId(), table.getParentName().getString())); + int indexToSkip = 0; + byte[][] dataPkValues = new byte[parentTable.getPKColumns().size()][]; + byte[][] localIndexPkValues = new byte[pkValues.length+1][]; + if (table.getBucketNum() != null) { + indexToSkip++; + } + if(table.getTenantId()!=null) { + indexToSkip++; + } + indexToSkip=+(table.getPKColumns().size()-parentTable.getPKColumns().size()); + + for(int i = indexToSkip; i< pkValues.length; i++) { + dataPkValues[i-indexToSkip] = pkValues[i]; + } + parentTable.newKey(ptr, dataPkValues); + HRegionLocation region = statement.getConnection().getQueryServices().getTableRegionLocation(parentTable.getTableName().getBytes(), ptr.get()); + byte[] regionPrefix = region.getRegionInfo().getStartKey().length==0?new byte[region.getRegionInfo().getEndKey().length]:region.getRegionInfo().getStartKey(); + localIndexPkValues[0] = regionPrefix; + for(int i=1;i indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes); TupleProjector tupleProjector = null; - Region dataRegion = null; byte[][] viewConstants = null; ColumnReference[] dataColumns = IndexUtil.deserializeDataTableColumnsToJoin(scan); @@ -138,13 +137,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) { 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 7eb1dc6..090f837 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 @@ -1342,7 +1342,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(); } @@ -2867,13 +2867,21 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso private List results = Lists.newArrayList(); private boolean allViewsNotInSingleRegion = false; + private PTable table; - private TableViewFinderResult(List results) { + private TableViewFinderResult(List 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() { 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 54c688a..111ec66 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 @@ -194,7 +194,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 indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes); indexMaintainer = indexMaintainers.get(0); 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 6b51138..3d7230b 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 @@ -37,8 +37,10 @@ import java.util.Set; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CoprocessorEnvironment; +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.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; @@ -80,8 +82,10 @@ import org.apache.phoenix.schema.PColumn; import org.apache.phoenix.schema.PRow; import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTableImpl; +import org.apache.phoenix.schema.PTableType; import org.apache.phoenix.schema.RowKeySchema; import org.apache.phoenix.schema.SortOrder; +import org.apache.phoenix.schema.PTable.IndexType; import org.apache.phoenix.schema.ValueSchema.Field; import org.apache.phoenix.schema.stats.StatisticsCollector; import org.apache.phoenix.schema.tuple.MultiKeyValueTuple; @@ -154,6 +158,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. @@ -169,6 +176,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{ Region region = c.getEnvironment().getRegion(); long ts = scan.getTimeRange().getMax(); StatisticsCollector stats = null; + boolean localIndexScan = ScanUtil.isLocalIndex(scan); if(ScanUtil.isAnalyzeTable(scan)) { byte[] gp_width_bytes = scan.getAttribute(BaseScannerRegionObserver.GUIDEPOST_WIDTH_BYTES); byte[] gp_per_region_bytes = scan.getAttribute(BaseScannerRegionObserver.GUIDEPOST_PER_REGION); @@ -176,7 +184,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{ stats = new StatisticsCollector(c.getEnvironment(), region.getRegionInfo().getTable().getNameAsString(), ts, gp_width_bytes, gp_per_region_bytes); } 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. @@ -213,6 +221,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{ byte[] upsertSelectTable = scan.getAttribute(BaseScannerRegionObserver.UPSERT_SELECT_TABLE); boolean isUpsert = false; boolean isDelete = false; + List cfsToDelete = new ArrayList(); byte[] deleteCQ = null; byte[] deleteCF = null; byte[] emptyCF = null; @@ -228,26 +237,39 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{ if (!isDelete) { deleteCF = scan.getAttribute(BaseScannerRegionObserver.DELETE_CF); deleteCQ = scan.getAttribute(BaseScannerRegionObserver.DELETE_CQ); + } else { + if (localIndexScan) { + for (HColumnDescriptor cf : region.getTableDesc().getFamilies()) { + if (cf.getNameAsString().startsWith( + QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) { + cfsToDelete.add(cf.getName()); + } + } + } else { + for (HColumnDescriptor cf : region.getTableDesc().getFamilies()) { + if (!cf.getNameAsString().startsWith( + QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) { + cfsToDelete.add(cf.getName()); + } + } + } } 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(c.getEnvironment()); 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); + c.getEnvironment().getRegion(), indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr); } if (j != null) { @@ -400,6 +422,9 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{ Cell firstKV = results.get(0); Delete delete = new Delete(firstKV.getRowArray(), firstKV.getRowOffset(), firstKV.getRowLength(),ts); + for(byte[] cf: cfsToDelete) { + delete.addFamily(cf); + } mutations.add(delete); } else if (isUpsert) { Arrays.fill(values, null); @@ -494,7 +519,7 @@ 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); } } catch (ConstraintViolationException e) { // Log and ignore in count @@ -536,7 +561,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{ } if (!indexMutations.isEmpty()) { - commitIndexMutations(c, region, indexMutations); + commitBatch(region, indexMutations, null); } final boolean hadAny = hasAny; @@ -586,30 +611,6 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{ return scanner; } - private void commitIndexMutations(final ObserverContext c, - Region region, List 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) { - ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionInfo().getRegionNameAsString(), - ie); - } finally { - if (table != null) table.close(); - } - } - indexMutations.clear(); - } - @Override public InternalScanner preCompact(ObserverContext c, final Store store, InternalScanner scanner, final ScanType scanType) 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..5d32c27 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 @@ -18,6 +18,7 @@ package org.apache.phoenix.hbase.index; import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy; +import org.apache.phoenix.query.QueryConstants; /** * Split policy for index regions to avoid split from external requests. @@ -26,11 +27,14 @@ public class IndexRegionSplitPolicy extends RegionSplitPolicy { @Override protected boolean shouldSplit() { - return false; - } - - protected boolean skipStoreFileRangeCheck() { return true; } - + + @Override + protected boolean skipStoreFileRangeCheck(String familyName) { + if (familyName.startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) { + return true; + } + return false; + } } 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 1c3d1e2..0b08068 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 @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Mutation; @@ -50,6 +49,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; @@ -93,6 +93,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"; @@ -134,6 +135,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. @@ -350,7 +352,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); @@ -373,7 +375,7 @@ public class Indexer extends BaseRegionObserver { super.postPut(e, put, edit, durability); return; } - doPost(edit, put, durability); + doPost(edit, put, durability, true); } @Override @@ -383,7 +385,7 @@ public class Indexer extends BaseRegionObserver { super.postDelete(e, delete, edit, durability); return; } - doPost(edit, delete, durability); + doPost(edit, delete, durability, true); } @Override @@ -398,12 +400,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); @@ -412,7 +414,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)) { @@ -446,16 +448,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> 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 { + try { + if (!ikv.getBatchFinished()) { + current.addTimelineAnnotation("Actually doing index update for first time"); + writer.writeAndKillYourselfOnFailure(indexUpdates, allowLocalUpdates); + } else if (allowLocalUpdates) { + Collection> localUpdates = + new ArrayList>(); + current.addTimelineAnnotation("Actually doing local index update for first time"); + for (Pair 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. @@ -518,7 +535,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); } @@ -542,7 +559,7 @@ public class Indexer extends BaseRegionObserver { * hopes they come up before the primary table finishes. */ Collection> indexUpdates = extractIndexUpdate(logEdit); - recoveryWriter.write(indexUpdates); + recoveryWriter.write(indexUpdates, true); } /** 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 deleted file mode 100644 index 146028e..0000000 --- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancer.java +++ /dev/null @@ -1,693 +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 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; - -/** - *

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.

- * - * 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.

- * - *

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.

- * - *

In case of balancing by table any table can balance first.

- * - */ - -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 userTableVsIndexTable = new HashMap(); - - Map indexTableVsUserTable = new HashMap(); - - /** - * Maintains colocation information of user regions and corresponding index regions. - */ - private Map> colocationInfo = - new ConcurrentHashMap>(); - - private Set balancedTables = new HashSet(); - - private boolean stopped = false; - - @Override - public void initialize() throws HBaseIOException { - Class 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> getColocationInfo() { - return colocationInfo; - } - - @Override - public void setMasterServices(MasterServices masterServices) { - this.master = masterServices; - } - - @Override - public List balanceCluster(Map> clusterState) - throws HBaseIOException { - synchronized (this.colocationInfo) { - boolean balanceByTable = conf.getBoolean("hbase.master.loadbalance.bytable", false); - List regionPlans = null; - - TableName tableName = null; - if (balanceByTable) { - Map tableKeys = null; - for (Entry> serverVsRegionList : clusterState - .entrySet()) { - ServerName sn = serverVsRegionList.getKey(); - List 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(); - 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> userClusterState = - new HashMap>(); - Map> indexClusterState = - new HashMap>(); - for (Entry> serverVsRegionList : clusterState - .entrySet()) { - ServerName sn = serverVsRegionList.getKey(); - List regionsInfos = serverVsRegionList.getValue(); - List idxRegionsToBeMoved = new ArrayList(); - List userRegionsToBeMoved = new ArrayList(); - 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 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(); - } else { - updateRegionPlans(regionPlans); - } - return prepareRegionPlansForClusterState(indexClusterState, regionPlans); - } - } - } - - private void updateServer(Map 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 prepareRegionPlansForClusterState( - Map> clusterState, List regionPlans) { - if (regionPlans == null) regionPlans = new ArrayList(); - ImmutableBytesWritable startKey = new ImmutableBytesWritable(); - for (Entry> serverVsRegionList : clusterState.entrySet()) { - List 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 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> roundRobinAssignment(List regions, - List servers) throws HBaseIOException { - List userRegions = new ArrayList(); - List indexRegions = new ArrayList(); - for (HRegionInfo hri : regions) { - seperateUserAndIndexRegion(hri, userRegions, indexRegions); - } - Map> 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 userRegions, - List indexRegions) { - if (indexTableVsUserTable.containsKey(hri.getTable())) { - indexRegions.add(hri); - return; - } - userRegions.add(hri); - } - - private Map> prepareIndexRegionsPlan( - List indexRegions, Map> bulkPlan, - List servers) throws HBaseIOException { - if (null != indexRegions && !indexRegions.isEmpty()) { - if (null == bulkPlan) { - bulkPlan = new ConcurrentHashMap>(); - } - for (HRegionInfo hri : indexRegions) { - if (LOG.isDebugEnabled()) { - LOG.debug("Preparing region plan for index region " - + hri.getRegionNameAsString() + '.'); - } - ServerName destServer = getDestServerForIdxRegion(hri); - List destServerRegions = null; - if (destServer == null) destServer = this.randomAssignment(hri, servers); - if (destServer != null) { - destServerRegions = bulkPlan.get(destServer); - if (null == destServerRegions) { - destServerRegions = new ArrayList(); - 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 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> bulkPlan) { - synchronized (this.colocationInfo) { - for (Entry> 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> retainAssignment( - Map regions, List servers) throws HBaseIOException { - Map userRegionsMap = - new ConcurrentHashMap(); - List indexRegions = new ArrayList(); - for (Entry e : regions.entrySet()) { - seperateUserAndIndexRegion(e, userRegionsMap, indexRegions, servers); - } - Map> 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 e, - Map userRegionsMap, List indexRegions, - List 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 immediateAssignment(List regions, - List servers) throws HBaseIOException { - return this.delegator.immediateAssignment(regions, servers); - } - - @Override - public ServerName randomAssignment(HRegionInfo regionInfo, List 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 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 onlineServers) { - TableName tableName = regionInfo.getTable(); - TableName mappedTable = getMappedTableToColocate(regionInfo.getTable()); - ImmutableBytesWritable startKey = new ImmutableBytesWritable(regionInfo.getStartKey()); - synchronized (this.colocationInfo) { - Map correspondingTableKeys = - this.colocationInfo.get(mappedTable); - Map 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 tabkeKeys = this.colocationInfo.get(tableName); - if (tabkeKeys == null) { - tabkeKeys = new ConcurrentHashMap(); - 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 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 tableDescriptors) { - HTableDescriptor desc = null; - for (Entry 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); - } - - /** - * 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 onlineRegions = regionStates.getRegionsOfTable(table); - for (HRegionInfo hri : onlineRegions) { - regionOnline(hri, regionStates.getRegionServerOfRegion(hri)); - } - Map regionsInTransition = regionStates.getRegionsInTransition(); - for (RegionState regionState : regionsInTransition.values()) { - if (table.equals(regionState.getRegion().getTable()) - && regionState.getServerName() != null) { - regionOnline(regionState.getRegion(), regionState.getServerName()); - } - } - } - } -} 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 deleted file mode 100644 index 1da5aff..0000000 --- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/master/IndexMasterObserver.java +++ /dev/null @@ -1,116 +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.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 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 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 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 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 = - region.getTable().getNameAsString() - .startsWith(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX) ? MetaDataUtil - .getUserTableName(tableName) : MetaDataUtil - .getLocalIndexTableName(tableName); - List 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 ctx, - TableName tableName) throws IOException { - if (balancer != null && balancer.isTableColocated(tableName)) { - balancer.removeTablesFromColocation(tableName); - } - } -} 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..aefe260 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 @@ -34,4 +34,7 @@ public interface IndexCommitter extends Stoppable { public void write(Multimap toWrite) throws IndexWriteException; + + public void write(Multimap toWrite, boolean allowLocalUpdates) + throws IndexWriteException; } \ No newline at end of file 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..5f497c6 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> indexUpdates) throws IOException { + public void writeAndKillYourselfOnFailure(Collection> indexUpdates, + boolean allowLocalUpdates) throws IOException { // convert the strings to htableinterfaces to which we can talk and group by TABLE Multimap 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 toWrite) throws IOException { + public void writeAndKillYourselfOnFailure(Multimap toWrite, + boolean allowLocalUpdates) throws IOException { try { - write(toWrite); + write(toWrite, allowLocalUpdates); if (LOG.isTraceEnabled()) { LOG.trace("Done writing all index updates!\n\t" + toWrite); } @@ -162,11 +164,13 @@ public class IndexWriter implements Stoppable { * We attempt to quickly determine if any write has failed and not write to the remaining indexes * to ensure a timely recovery of the failed index writes. * @param toWrite Updates to write + * @param allowLocalUpdates whether to write local index updates or not. * @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> toWrite) throws IndexWriteException { - write(resolveTableReferences(toWrite)); + public void write(Collection> toWrite, boolean allowLocalUpdates) + throws IndexWriteException { + write(resolveTableReferences(toWrite), allowLocalUpdates); } /** @@ -181,6 +185,16 @@ public class IndexWriter implements Stoppable { /** + * see {@link #write(Collection)} + * @param toWrite + * @throws IndexWriteException + */ + public void write(Multimap 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 * @return pairs that can then be written by an {@link IndexWriter}. 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 56bf637..96cbb7c 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 @@ -23,10 +23,11 @@ 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; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.phoenix.hbase.index.exception.IndexWriteException; import org.apache.phoenix.hbase.index.exception.SingleIndexWriteFailureException; import org.apache.phoenix.hbase.index.parallel.EarlyExitFailure; import org.apache.phoenix.hbase.index.parallel.QuickFailingTaskRunner; @@ -38,8 +39,6 @@ import org.apache.phoenix.hbase.index.table.CachingHTableFactory; import org.apache.phoenix.hbase.index.table.HTableFactory; import org.apache.phoenix.hbase.index.table.HTableInterfaceReference; import org.apache.phoenix.hbase.index.util.KeyValueBuilder; -import org.apache.phoenix.util.IndexUtil; -import org.apache.phoenix.util.MetaDataUtil; import com.google.common.collect.Multimap; @@ -98,7 +97,14 @@ public class ParallelWriterIndexCommitter implements IndexCommitter { } @Override - public void write(Multimap toWrite) throws SingleIndexWriteFailureException { + public void write(Multimap toWrite) + throws IndexWriteException { + write(toWrite, false); + } + + @Override + public void write(Multimap 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 @@ -150,14 +156,20 @@ public class ParallelWriterIndexCommitter implements IndexCommitter { // index is pretty hacky. If we're going to keep this, we should revisit that // as well. try { - if (tableReference.getTableName().startsWith(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX)) { - Region indexRegion = IndexUtil.getIndexRegion(env); - if (indexRegion != null) { - throwFailureIfDone(); - indexRegion.batchMutate(mutations.toArray(new Mutation[mutations.size()]), + if (env != null + && tableReference.getTableName().equals( + env.getRegion().getTableDesc().getNameAsString())) { + throwFailureIfDone(); + + if(allowLocalUpdates) { + for (Mutation m : mutations) { + m.setDurability(Durability.SKIP_WAL); + } + env.getRegion().batchMutate( + mutations.toArray(new Mutation[mutations.size()]), HConstants.NO_NONCE, HConstants.NO_NONCE); - return null; } + return null; } } catch (IOException ignord) { // when it's failed we fall back to the standard & slow way @@ -200,7 +212,7 @@ public class ParallelWriterIndexCommitter implements IndexCommitter { } } - + private void propagateFailure(Throwable throwable) throws SingleIndexWriteFailureException { try { throw throwable; 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 b1b2656..571ca3d 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,11 +25,13 @@ 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; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.phoenix.hbase.index.CapturingAbortable; +import org.apache.phoenix.hbase.index.exception.IndexWriteException; import org.apache.phoenix.hbase.index.exception.MultiIndexWriteFailureException; import org.apache.phoenix.hbase.index.exception.SingleIndexWriteFailureException; import org.apache.phoenix.hbase.index.parallel.EarlyExitFailure; @@ -110,7 +112,14 @@ public class TrackingParallelWriterIndexCommitter implements IndexCommitter { } @Override - public void write(Multimap toWrite) throws MultiIndexWriteFailureException { + public void write(Multimap toWrite) + throws IndexWriteException { + write(toWrite, false); + } + + @Override + public void write(Multimap toWrite, + final boolean allowLocalUpdates) throws MultiIndexWriteFailureException { Set>> entries = toWrite.asMap().entrySet(); TaskBatch tasks = new TaskBatch(entries.size()); List tables = new ArrayList(entries.size()); @@ -154,14 +163,18 @@ public class TrackingParallelWriterIndexCommitter implements IndexCommitter { // 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 (tableReference.getTableName().startsWith(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX)) { - Region indexRegion = IndexUtil.getIndexRegion(env); - if (indexRegion != null) { - throwFailureIfDone(); - indexRegion.batchMutate(mutations.toArray(new Mutation[mutations.size()]), + if (tableReference.getTableName().equals( + env.getRegion().getTableDesc().getNameAsString())) { + throwFailureIfDone(); + if(allowLocalUpdates) { + for (Mutation m : mutations) { + m.setDurability(Durability.SKIP_WAL); + } + env.getRegion().batchMutate( + mutations.toArray(new Mutation[mutations.size()]), HConstants.NO_NONCE, HConstants.NO_NONCE); - return Boolean.TRUE; - } + } + return Boolean.TRUE; } } catch (IOException ignord) { // when it's failed we fall back to the standard & slow way 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 b060345..1fb6d73 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 @@ -237,6 +237,7 @@ public class IndexMaintainer implements Writable, Iterable { // columns required to evaluate all expressions in indexedExpressions (this does not include columns in the data row key) private Set indexedColumns; private Set coveredColumns; + private Map coveredColumnsMap; // columns required to create index row i.e. indexedColumns + coveredColumns (this does not include columns in the data row key) private Set allColumns; // TODO remove this in the next major release @@ -325,6 +326,7 @@ public class IndexMaintainer implements Writable, Iterable { this.indexedColumnTypes = Lists.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); @@ -392,6 +394,14 @@ public class IndexMaintainer implements Writable, Iterable { 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); @@ -807,7 +817,7 @@ public class IndexMaintainer implements Writable, Iterable { put.add(kvBuilder.buildPut(new ImmutableBytesPtr(indexRowKey), this.getEmptyKeyValueFamily(), QueryConstants.EMPTY_COLUMN_BYTES_PTR, ts, ByteUtil.EMPTY_BYTE_ARRAY_PTR)); - put.setDurability(!indexWALDisabled ? Durability.USE_DEFAULT : Durability.SKIP_WAL); + put.setDurability(!indexWALDisabled? Durability.USE_DEFAULT : Durability.SKIP_WAL); } int i = 0; for (ColumnReference ref : this.getCoverededColumns()) { @@ -821,7 +831,12 @@ public class IndexMaintainer implements Writable, Iterable { 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; @@ -895,7 +910,13 @@ public class IndexMaintainer implements Writable, Iterable { delete = new Delete(indexRowKey); delete.setDurability(!indexWALDisabled ? Durability.USE_DEFAULT : Durability.SKIP_WAL); } - delete.deleteColumns(ref.getFamily(), IndexUtil.getIndexColumnName(ref.getFamily(), ref.getQualifier()), ts); + if(this.isLocalIndex) { + ColumnReference columnReference = this.coveredColumnsMap.get(ref); + delete.deleteColumns(columnReference.getFamily(), IndexUtil.getIndexColumnName( + ref.getFamily(), ref.getQualifier()), ts); + } else { + delete.deleteColumns(ref.getFamily(), IndexUtil.getIndexColumnName(ref.getFamily(), ref.getQualifier()), ts); + } } } } @@ -949,10 +970,15 @@ public class IndexMaintainer implements Writable, Iterable { 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); @@ -1349,15 +1375,16 @@ public class IndexMaintainer implements Writable, Iterable { return allColumns.iterator(); } - public ValueGetter createGetterFromKeyValues(final byte[] rowKey, Collection pendingUpdates) { + public ValueGetter createGetterFromKeyValues(final byte[] rowKey, + Collection pendingUpdates) { final Map valueMap = Maps.newHashMapWithExpectedSize(pendingUpdates .size()); for (Cell kv : pendingUpdates) { // create new pointers to each part of the kv - ImmutableBytesPtr family = new ImmutableBytesPtr(kv.getRowArray(),kv.getFamilyOffset(),kv.getFamilyLength()); - ImmutableBytesPtr qual = new ImmutableBytesPtr(kv.getRowArray(), kv.getQualifierOffset(), kv.getQualifierLength()); ImmutableBytesPtr value = new ImmutableBytesPtr(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()); - valueMap.put(new ColumnReference(kv.getRowArray(), kv.getFamilyOffset(), kv.getFamilyLength(), kv.getRowArray(), kv.getQualifierOffset(), kv.getQualifierLength()), value); + valueMap.put( + new ColumnReference(kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), + kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()), value); } return new ValueGetter() { @Override 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 222aefb..06c951b 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 @@ -20,6 +20,7 @@ package org.apache.phoenix.index; import java.io.IOException; import java.sql.SQLException; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -27,6 +28,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.phoenix.cache.GlobalCache; import org.apache.phoenix.cache.IndexMetaDataCache; @@ -45,6 +47,7 @@ import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr; import org.apache.phoenix.hbase.index.util.IndexManagementUtil; import org.apache.phoenix.hbase.index.util.KeyValueBuilder; import org.apache.phoenix.hbase.index.write.IndexWriter; +import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.util.MetaDataUtil; import org.apache.phoenix.util.PhoenixRuntime; import org.apache.phoenix.util.ServerUtil; @@ -134,7 +137,7 @@ public class PhoenixIndexCodec extends BaseIndexCodec { // TODO: state.getCurrentRowKey() should take an ImmutableBytesWritable arg to prevent byte copy byte[] dataRowKey = state.getCurrentRowKey(); ptr.set(dataRowKey); - byte[] localIndexTableName = MetaDataUtil.getLocalIndexPhysicalName(env.getRegion().getTableDesc().getName()); + byte[] dataTableName = env.getRegion().getTableDesc().getName(); ValueGetter valueGetter = null; Scanner scanner = null; for (IndexMaintainer maintainer : indexMaintainers) { @@ -148,11 +151,12 @@ public class PhoenixIndexCodec extends BaseIndexCodec { if (maintainer.isImmutableRows()) { indexUpdate = new IndexUpdate(new ColumnTracker(maintainer.getAllColumns())); if(maintainer.isLocalIndex()) { - indexUpdate.setTable(localIndexTableName); + indexUpdate.setTable(dataTableName); } else { indexUpdate.setTable(maintainer.getIndexTableName()); } - valueGetter = maintainer.createGetterFromKeyValues(dataRowKey, state.getPendingUpdate()); + valueGetter = maintainer.createGetterFromKeyValues(dataRowKey, + state.getPendingUpdate()); } else { // TODO: if more efficient, I could do this just once with all columns in all indexes Pair statePair = state.getIndexedColumnsTableState(maintainer.getAllColumns()); diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java index 20f05ff..10d1911 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java @@ -22,8 +22,11 @@ import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.concurrent.ExecutionException; @@ -317,9 +320,14 @@ public class CsvBulkLoadTool extends Configured implements Tool { * @throws Exception */ private void completebulkload(Configuration conf,Path outputPath , List tablesToBeLoaded) throws Exception { - for(TargetTableRef table : tablesToBeLoaded) { + TargetTableRef targetTableRef = tablesToBeLoaded.get(0); + Set set = new HashSet(); + for(TargetTableRef table : tablesToBeLoaded) { + set.add(table.getPhysicalName()); + } + + for(String tableName : set) { LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf); - String tableName = table.getPhysicalName(); Path tableOutputPath = new Path(outputPath,tableName); HTable htable = new HTable(conf,tableName); LOG.info("Loading HFiles for {} from {}", tableName , tableOutputPath); @@ -456,16 +464,8 @@ public class CsvBulkLoadTool extends Configured implements Tool { PTable table = PhoenixRuntime.getTable(conn, qualifiedTableName); List indexTables = new ArrayList(); for(PTable indexTable : table.getIndexes()){ - if (indexTable.getIndexType() == IndexType.LOCAL) { - throw new UnsupportedOperationException("Local indexes not supported by CSV Bulk Loader"); - /*indexTables.add( - new TargetTableRef(getQualifiedTableName(schemaName, - indexTable.getTableName().getString()), - MetaDataUtil.getLocalIndexTableName(qualifiedTableName))); */ - } else { - indexTables.add(new TargetTableRef(getQualifiedTableName(schemaName, - indexTable.getTableName().getString()))); - } + indexTables.add(new TargetTableRef(getQualifiedTableName(schemaName, indexTable + .getTableName().getString()), indexTable.getPhysicalName().getString())); } return indexTables; } 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 8a4f963..0943b7a 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 @@ -199,7 +199,7 @@ public class IndexTool extends Configured implements Tool { // computed from the qDataTable name. String physicalIndexTable = qIndexTable; if (IndexType.LOCAL.equals(pindexTable.getIndexType())) { - physicalIndexTable = MetaDataUtil.getLocalIndexTableName(qDataTable); + physicalIndexTable = qDataTable; } final PhoenixConnection pConnection = connection.unwrap(PhoenixConnection.class); diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java index 3b53309..392a5a2 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java @@ -68,7 +68,7 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated public HTableDescriptor getTableDescriptor(byte[] tableName) throws SQLException; public List getAllTableRegions(byte[] tableName) throws SQLException; - + public HRegionLocation getTableRegionLocation(byte[] tableName, byte[] row) throws SQLException; public PhoenixConnection connect(String url, Properties info) throws SQLException; public MetaDataMutationResult getTable(PName tenantId, byte[] schemaName, byte[] tableName, long tableTimestamp, long clientTimetamp) throws SQLException; 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 b718089..62d7ff8 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 @@ -27,9 +27,11 @@ import static org.apache.phoenix.util.UpgradeUtil.upgradeTo4_5_0; import java.io.IOException; import java.sql.SQLException; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -66,7 +68,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; @@ -457,6 +458,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement } @Override + public HRegionLocation getTableRegionLocation(byte[] tableName, byte[] row) throws SQLException { + /* + * Use HConnection.getRegionLocation as it uses the cache in HConnection, to get the region + * to which specified row belongs to. + */ + int retryCount = 0, maxRetryCount = 1; + boolean reload =false; + while (true) { + try { + return connection.getRegionLocation(TableName.valueOf(tableName), row, reload); + } catch (org.apache.hadoop.hbase.TableNotFoundException e) { + String fullName = Bytes.toString(tableName); + throw new TableNotFoundException(SchemaUtil.getSchemaNameFromFullName(fullName), SchemaUtil.getTableNameFromFullName(fullName)); + } catch (IOException e) { + if (retryCount++ < maxRetryCount) { // One retry, in case split occurs while navigating + reload = true; + continue; + } + throw new SQLExceptionInfo.Builder(SQLExceptionCode.GET_TABLE_REGIONS_FAIL) + .setRootCause(e).build().buildException(); + } + } + } + + @Override public PMetaData addTable(PTable table) throws SQLException { synchronized (latestMetaDataLock) { try { @@ -691,18 +717,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 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; + } } } @@ -837,10 +859,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement HTableDescriptor newDesc = generateTableDescriptor(tableName, existingDesc, tableType , props, families, splits); if (!tableExist) { - if (newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null && Boolean.TRUE.equals( - PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) { - newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName()); - } // Remove the splitPolicy attribute to prevent HBASE-12570 if (isMetaTable) { newDesc.remove(HTableDescriptor.SPLIT_POLICY); @@ -878,11 +896,16 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement if (isMetaTable) { checkClientServerCompatibility(); } + for(Pair> family: families) { + if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) { + newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName()); + break; + } + } if (!modifyExistingMetaData || existingDesc.equals(newDesc)) { return existingDesc; } - modifyTable(tableName, newDesc, true); return newDesc; } @@ -1089,53 +1112,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement } } - private void ensureLocalIndexTableCreated(byte[] physicalTableName, Map tableProps, List>> families, byte[][] splits, long timestamp) throws SQLException { - PTable table; - String parentTableName = Bytes.toString(physicalTableName, MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX_BYTES.length, - physicalTableName.length - MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX_BYTES.length); - try { - synchronized (latestMetaDataLock) { - throwConnectionClosedIfNullMetaData(); - table = latestMetaData.getTable(new PTableKey(PName.EMPTY_NAME, parentTableName)); - 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); - } - - private void ensureLocalIndexTableCreated(byte[] physicalTableName, Map tableProps, List>> families, byte[][] splits) 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); - 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; @@ -1172,25 +1148,29 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement } private boolean ensureLocalIndexTableDropped(byte[] physicalTableName, long timestamp) throws SQLException { - byte[] physicalIndexName = MetaDataUtil.getLocalIndexPhysicalName(physicalTableName); HTableDescriptor desc = null; HBaseAdmin admin = null; boolean wasDeleted = false; try { - admin = new HBaseAdmin(config); + admin = this.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)); + 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) { - admin.disableTable(physicalIndexName); - admin.deleteTable(physicalIndexName); - clearTableRegionCache(physicalIndexName); + List columnFamiles = new ArrayList(); + 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 } @@ -1219,7 +1199,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement 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)) { + 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); @@ -1229,9 +1209,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)); - } 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)); } } @@ -1349,13 +1327,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement dropTables(result.getTableNamesToDelete()); } invalidateTables(result.getTableNamesToDelete()); + long timestamp = MetaDataUtil.getClientTimeStamp(tableMetaData); if (tableType == PTableType.TABLE) { byte[] physicalName = SchemaUtil.getTableNameAsBytes(schemaBytes, tableBytes); - long timestamp = MetaDataUtil.getClientTimeStamp(tableMetaData); ensureViewIndexTableDropped(physicalName, timestamp); ensureLocalIndexTableDropped(physicalName, timestamp); tableStatsCache.invalidate(new ImmutableBytesPtr(physicalName)); - } + } break; default: break; diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java index 3fa0c1e..10539a2 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java @@ -141,6 +141,21 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple } @Override + public HRegionLocation getTableRegionLocation(byte[] tableName, byte[] row) throws SQLException { + List regions = tableSplits.get(Bytes.toString(tableName)); + if (regions != null) { + for(HRegionLocation region: regions) { + if (Bytes.compareTo(region.getRegionInfo().getStartKey(), row) <= 0 + && Bytes.compareTo(region.getRegionInfo().getEndKey(), row) > 0) { + return region; + } + } + } + return new HRegionLocation( + new HRegionInfo(TableName.valueOf(tableName), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW), + SERVER_NAME, -1); + } + @Override public PMetaData addTable(PTable table) throws SQLException { return metaData = metaData.addTable(table); } diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java index 4153652..31b959b 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java @@ -74,6 +74,10 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple } @Override + public HRegionLocation getTableRegionLocation(byte[] tableName, byte[] row) throws SQLException { + return getDelegate().getTableRegionLocation(tableName, row); + } + @Override public PMetaData addTable(PTable table) throws SQLException { return getDelegate().addTable(table); } 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 268bfc1..dedc9ba 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 @@ -112,6 +112,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.util.Bytes; import org.apache.phoenix.coprocessor.MetaDataProtocol; import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr; +import org.apache.phoenix.hbase.index.util.VersionUtil; import org.apache.phoenix.schema.MetaDataSplitPolicy; import org.apache.phoenix.schema.PName; import org.apache.phoenix.schema.PNameFactory; @@ -158,6 +159,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. @@ -181,6 +183,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"; 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 77ca2c3..4e4a0c8 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 @@ -104,6 +104,7 @@ import java.util.Iterator; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.NavigableSet; import java.util.Properties; import java.util.Set; @@ -830,11 +831,6 @@ public class MetaDataClient { MetaDataUtil.getViewIndexSchemaName(table.getSchemaName().getString()), MetaDataUtil.getViewIndexTableName(table.getTableName().getString())))); } - if (MetaDataUtil.hasLocalIndexTable(connection, table.getName())) { - names.add(PNameFactory.newName(SchemaUtil.getTableName( - MetaDataUtil.getLocalIndexSchemaName(table.getSchemaName().getString()), - MetaDataUtil.getLocalIndexTableName(table.getTableName().getString())))); - } for (final PName name : names) { PTable indexLogicalTable = new DelegateTable(table) { @@ -1280,7 +1276,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())); } } @@ -1525,21 +1521,27 @@ public class MetaDataClient { List physicalNames = Collections.emptyList(); boolean addSaltColumn = false; boolean rowKeyOrderOptimizable = true; + boolean isLocalIndex = indexType == IndexType.LOCAL; if (parent != null && tableType == PTableType.INDEX) { // Index on view // 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); - defaultFamilyName = parent.getDefaultFamilyName() == null ? null : parent.getDefaultFamilyName().getString(); - if (indexType == IndexType.LOCAL) { + addSaltColumn = (saltBucketNum != null && !isLocalIndex); + 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()))); } @@ -1802,7 +1804,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 @@ -1852,7 +1856,9 @@ public class MetaDataClient { throwIfInsufficientColumns(schemaName, tableName, pkColumns, saltBucketNum!=null, multiTenant); for (PName familyName : familyNames.values()) { - Collection> props = statement.getProps().get(familyName.getString()); + String fam = familyName.getString(); + Collection> props = + statement.getProps().get(IndexUtil.getActualColumnFamilyName(fam)); if (props.isEmpty()) { familyPropList.add(new Pair>(familyName.getBytes(),commonFamilyProps)); } else { @@ -1873,7 +1879,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 ? (indexType != IndexType.LOCAL ? QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES + : QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES) + : Bytes.toBytes(defaultFamilyName); familyPropList.add(new Pair>(cf, commonFamilyProps)); } @@ -2003,9 +2012,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 { splits = SchemaUtil.processSplits(splits, pkColumns, saltBucketNum, connection.getQueryServices().getProps().getBoolean( QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, QueryServicesOptions.DEFAULT_FORCE_ROW_KEY_ORDER)); @@ -2209,7 +2216,6 @@ public class MetaDataClient { default: connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, tableName), parentTableName, result.getMutationTime()); - if (result.getTable() != null && tableType != PTableType.VIEW) { connection.setAutoCommit(true); PTable table = result.getTable(); @@ -2237,19 +2243,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. 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? @@ -2285,6 +2278,10 @@ public class MetaDataClient { buf.append("'" + ref.getTable().getName().getString() + "',"); } buf.setCharAt(buf.length() - 1, ')'); + if(tableRefs.get(0).getTable().getIndexType()==IndexType.LOCAL) { + buf.append(" AND COLUMN_FAMILY='" + QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX + + QueryConstants.DEFAULT_COLUMN_FAMILY + "'"); + } conn.createStatement().execute(buf.toString()); success = true; } catch (SQLException e) { 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 3bf6f23..a1e3891 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 @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; @@ -87,6 +88,9 @@ import org.apache.phoenix.schema.types.PVarchar; import com.google.common.collect.Lists; +import static org.apache.phoenix.query.QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX; +import static org.apache.phoenix.query.QueryConstants.NAME_SEPARATOR;; + public class IndexUtil { public static final String INDEX_COLUMN_NAME_SEP = ":"; public static final byte[] INDEX_COLUMN_NAME_SEP_BYTES = Bytes.toBytes(INDEX_COLUMN_NAME_SEP); @@ -137,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) { @@ -169,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); } @@ -260,7 +277,14 @@ public class IndexUtil { } }; - indexMutations.add(maintainer.buildUpdateMutation(kvBuilder, valueGetter, ptr, ts, null, null)); + byte[] regionStartKey = null; + byte[] regionEndkey = null; + if(maintainer.isLocalIndex()) { + HRegionLocation tableRegionLocation = connection.getQueryServices().getTableRegionLocation(table.getTableName().getBytes(), dataMutation.getRow()); + regionStartKey = tableRegionLocation.getRegionInfo().getStartKey(); + regionEndkey = tableRegionLocation.getRegionInfo().getEndKey(); + } + indexMutations.add(maintainer.buildUpdateMutation(kvBuilder, valueGetter, ptr, ts, regionStartKey, regionEndkey)); } } return indexMutations; @@ -305,55 +329,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 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 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; 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 ee73a58..7ec0542 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 @@ -26,6 +26,7 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; @@ -357,15 +358,24 @@ 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 void deleteViewIndexSequences(PhoenixConnection connection, PName name) throws SQLException { String schemaName = getViewIndexSchemaName(name); connection.createStatement().executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_ESCAPED + diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java index 5414d4f..ee992a9 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java @@ -54,6 +54,7 @@ import org.apache.phoenix.schema.PDatum; import org.apache.phoenix.schema.PMetaData; import org.apache.phoenix.schema.PName; import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.schema.PTable.IndexType; import org.apache.phoenix.schema.RowKeySchema; import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder; import org.apache.phoenix.schema.SaltingUtil; @@ -382,18 +383,18 @@ public class SchemaUtil { } public static byte[] getEmptyColumnFamily(PName defaultColumnFamily, List families) { - return families.isEmpty() ? defaultColumnFamily == null ? QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES : defaultColumnFamily.getBytes() : families.get(0).getName().getBytes(); + return families.isEmpty() ? defaultColumnFamily == null ? QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES : defaultColumnFamily.getBytes() : families.get(0).getName().getBytes(); } public static byte[] getEmptyColumnFamily(PTable table) { - List families = table.getColumnFamilies(); - return families.isEmpty() ? table.getDefaultFamilyName() == null ? QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES : table.getDefaultFamilyName().getBytes() : families.get(0).getName().getBytes(); + return getEmptyColumnFamily(table.getDefaultFamilyName(), table.getColumnFamilies()); } public static ImmutableBytesPtr getEmptyColumnFamilyPtr(PTable table) { List families = table.getColumnFamilies(); - return families.isEmpty() ? table.getDefaultFamilyName() == null ? QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES_PTR : table.getDefaultFamilyName().getBytesPtr() : families.get(0) - .getName().getBytesPtr(); + boolean isLocalIndex = table.getIndexType()==IndexType.LOCAL; + return families.isEmpty() ? table.getDefaultFamilyName() == null ?(isLocalIndex?QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES_PTR: QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES_PTR ): table.getDefaultFamilyName().getBytesPtr() : families.get(0) + .getName().getBytesPtr(); } public static boolean isMetaTable(byte[] tableName) { diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java index 8f576cf..d76c183 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java @@ -121,7 +121,7 @@ public class TestIndexWriter { KillServerOnFailurePolicy policy = new KillServerOnFailurePolicy(); policy.setup(stop, abort); IndexWriter writer = new IndexWriter(committer, policy); - writer.write(indexUpdates); + writer.write(indexUpdates, false); assertTrue("Writer returned before the table batch completed! Likely a race condition tripped", completed[0]); writer.stop(this.testName.getTableNameString() + " finished"); @@ -195,7 +195,7 @@ public class TestIndexWriter { policy.setup(stop, abort); IndexWriter writer = new IndexWriter(committer, policy); try { - writer.write(indexUpdates); + writer.write(indexUpdates, false); fail("Should not have successfully completed all index writes"); } catch (SingleIndexWriteFailureException e) { LOG.info("Correctly got a failure to reach the index", e); @@ -268,7 +268,7 @@ public class TestIndexWriter { @Override public void run() { try { - writer.write(indexUpdates); + writer.write(indexUpdates, false); } catch (IndexWriteException e) { failedWrite[0] = true; } 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 5265afc..403f488 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 @@ -118,12 +118,10 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.client.HBaseAdmin; -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.regionserver.LocalIndexMerger; import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.util.Bytes; import org.apache.phoenix.end2end.BaseClientManagedTimeIT; @@ -654,8 +652,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.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);