Skip to content

Commit

Permalink
PHOENIX-6129 : Optimize tableExists() call while retrieving correct M…
Browse files Browse the repository at this point in the history
…UTEX table (#920)
  • Loading branch information
virajjasani committed Oct 26, 2020
1 parent 601375e commit b9fb461
Show file tree
Hide file tree
Showing 2 changed files with 61 additions and 31 deletions.
Expand Up @@ -4359,19 +4359,12 @@ void ensureSystemTablesMigratedToSystemNamespace()
* making use of HBase's checkAndPut api.
*
* @return true if client won the race, false otherwise
* @throws IOException
* @throws SQLException
*/
@VisibleForTesting
public boolean acquireUpgradeMutex(long currentServerSideTableTimestamp)
throws IOException,
SQLException {
throws SQLException {
Preconditions.checkArgument(currentServerSideTableTimestamp < MIN_SYSTEM_TABLE_TIMESTAMP);
byte[] sysMutexPhysicalTableNameBytes = getSysMutexPhysicalTableNameBytes();
if(sysMutexPhysicalTableNameBytes == null) {
throw new UpgradeInProgressException(getVersion(currentServerSideTableTimestamp),
getVersion(MIN_SYSTEM_TABLE_TIMESTAMP));
}
if (!writeMutexCell(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE, null, null)) {
throw new UpgradeInProgressException(getVersion(currentServerSideTableTimestamp),
Expand All @@ -4384,15 +4377,13 @@ public boolean acquireUpgradeMutex(long currentServerSideTableTimestamp)
public boolean writeMutexCell(String tenantId, String schemaName, String tableName,
String columnName, String familyName) throws SQLException {
try {
byte[] rowKey =
columnName != null
? SchemaUtil.getColumnKey(tenantId, schemaName, tableName, columnName,
familyName)
: SchemaUtil.getTableKey(tenantId, schemaName, tableName);
byte[] rowKey = columnName != null
? SchemaUtil.getColumnKey(tenantId, schemaName, tableName,
columnName, familyName)
: SchemaUtil.getTableKey(tenantId, schemaName, tableName);
// at this point the system mutex table should have been created or
// an exception thrown
byte[] sysMutexPhysicalTableNameBytes = getSysMutexPhysicalTableNameBytes();
try (Table sysMutexTable = getTable(sysMutexPhysicalTableNameBytes)) {
try (Table sysMutexTable = getSysMutexTable()) {
byte[] family = PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES;
byte[] qualifier = PhoenixDatabaseMetaData.SYSTEM_MUTEX_COLUMN_NAME_BYTES;
byte[] value = MUTEX_LOCKED;
Expand Down Expand Up @@ -4428,15 +4419,13 @@ public void releaseUpgradeMutex() throws IOException, SQLException {
public void deleteMutexCell(String tenantId, String schemaName, String tableName,
String columnName, String familyName) throws SQLException {
try {
byte[] rowKey =
columnName != null
? SchemaUtil.getColumnKey(tenantId, schemaName, tableName, columnName,
familyName)
: SchemaUtil.getTableKey(tenantId, schemaName, tableName);
byte[] rowKey = columnName != null
? SchemaUtil.getColumnKey(tenantId, schemaName, tableName,
columnName, familyName)
: SchemaUtil.getTableKey(tenantId, schemaName, tableName);
// at this point the system mutex table should have been created or
// an exception thrown
byte[] sysMutexPhysicalTableNameBytes = getSysMutexPhysicalTableNameBytes();
try (Table sysMutexTable = getTable(sysMutexPhysicalTableNameBytes)) {
try (Table sysMutexTable = getSysMutexTable()) {
byte[] family = PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES;
byte[] qualifier = PhoenixDatabaseMetaData.SYSTEM_MUTEX_COLUMN_NAME_BYTES;
Delete delete = new Delete(rowKey);
Expand All @@ -4454,17 +4443,18 @@ public void deleteMutexCell(String tenantId, String schemaName, String tableName
}
}

private byte[] getSysMutexPhysicalTableNameBytes() throws IOException, SQLException {
byte[] sysMutexPhysicalTableNameBytes = null;
try(Admin admin = getAdmin()) {
if(admin.tableExists(PhoenixDatabaseMetaData.SYSTEM_MUTEX_HBASE_TABLE_NAME)) {
sysMutexPhysicalTableNameBytes = PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME_BYTES;
} else if (admin.tableExists(TableName.valueOf(
SchemaUtil.getPhysicalTableName(SYSTEM_MUTEX_NAME, props).getName()))) {
sysMutexPhysicalTableNameBytes = SchemaUtil.getPhysicalTableName(SYSTEM_MUTEX_NAME, props).getName();
@VisibleForTesting
public Table getSysMutexTable() throws SQLException, IOException {
String table = SYSTEM_MUTEX_NAME;
TableName tableName = TableName.valueOf(table);
try (Admin admin = getAdmin()) {
if (!admin.tableExists(tableName)) {
table = table.replace(QueryConstants.NAME_SEPARATOR,
QueryConstants.NAMESPACE_SEPARATOR);
tableName = TableName.valueOf(table);
}
return connection.getTable(tableName);
}
return sysMutexPhysicalTableNameBytes;
}

private String addColumn(String columnsToAddSoFar, String columns) {
Expand Down
Expand Up @@ -23,6 +23,7 @@
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TTL_FOR_MUTEX;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
Expand All @@ -46,7 +47,9 @@
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.phoenix.exception.PhoenixIOException;
Expand Down Expand Up @@ -78,6 +81,12 @@ public class ConnectionQueryServicesImplTest {
@Mock
private ReadOnlyProps readOnlyProps;

@Mock
private Connection mockConn;

@Mock
private Table mockTable;

public static final TableDescriptorBuilder SYS_TASK_TDB = TableDescriptorBuilder
.newBuilder(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_TASK_NAME));
public static final TableDescriptorBuilder SYS_TASK_TDB_SP = TableDescriptorBuilder
Expand All @@ -93,12 +102,17 @@ public void setup() throws IOException, NoSuchFieldException,
.getDeclaredField("props");
props.setAccessible(true);
props.set(mockCqs, readOnlyProps);
props = ConnectionQueryServicesImpl.class.getDeclaredField("connection");
props.setAccessible(true);
props.set(mockCqs, mockConn);
when(mockCqs.checkIfSysMutexExistsAndModifyTTLIfRequired(mockAdmin))
.thenCallRealMethod();
when(mockCqs.updateAndConfirmSplitPolicyForTask(SYS_TASK_TDB))
.thenCallRealMethod();
when(mockCqs.updateAndConfirmSplitPolicyForTask(SYS_TASK_TDB_SP))
.thenCallRealMethod();
when(mockCqs.getSysMutexTable()).thenCallRealMethod();
when(mockCqs.getAdmin()).thenCallRealMethod();
}

@SuppressWarnings("unchecked")
Expand Down Expand Up @@ -193,4 +207,30 @@ public void testSysTaskSplitPolicyWithError() {
e.getMessage());
}
}

@Test
public void testGetSysMutexTableWithName() throws Exception {
when(mockAdmin.tableExists(any())).thenReturn(true);
when(mockConn.getAdmin()).thenReturn(mockAdmin);
when(mockConn.getTable(TableName.valueOf("SYSTEM.MUTEX")))
.thenReturn(mockTable);
assertSame(mockCqs.getSysMutexTable(), mockTable);
verify(mockAdmin, Mockito.times(1)).tableExists(any());
verify(mockConn, Mockito.times(1)).getAdmin();
verify(mockConn, Mockito.times(1))
.getTable(TableName.valueOf("SYSTEM.MUTEX"));
}

@Test
public void testGetSysMutexTableWithNamespace() throws Exception {
when(mockAdmin.tableExists(any())).thenReturn(false);
when(mockConn.getAdmin()).thenReturn(mockAdmin);
when(mockConn.getTable(TableName.valueOf("SYSTEM:MUTEX")))
.thenReturn(mockTable);
assertSame(mockCqs.getSysMutexTable(), mockTable);
verify(mockAdmin, Mockito.times(1)).tableExists(any());
verify(mockConn, Mockito.times(1)).getAdmin();
verify(mockConn, Mockito.times(1))
.getTable(TableName.valueOf("SYSTEM:MUTEX"));
}
}

0 comments on commit b9fb461

Please sign in to comment.