in phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java [1142:1707]
private PTable getTableFromCells(List<Cell> tableCellList, List<List<Cell>> allColumnCellList,
long clientTimeStamp, int clientVersion, PTable oldTable)
throws IOException, SQLException {
Cell[] tableKeyValues = new Cell[TABLE_KV_COLUMNS.size()];
Cell[] colKeyValues = new Cell[COLUMN_KV_COLUMNS.size()];
// Create PTable based on KeyValues from scan
Cell keyValue = tableCellList.get(0);
byte[] keyBuffer = keyValue.getRowArray();
int keyLength = keyValue.getRowLength();
int keyOffset = keyValue.getRowOffset();
PName tenantId = newPName(keyBuffer, keyOffset, keyLength);
int tenantIdLength = (tenantId == null) ? 0 : tenantId.getBytes().length;
if (tenantIdLength == 0) {
tenantId = null;
}
PName schemaName = newPName(keyBuffer, keyOffset + tenantIdLength + 1, keyLength);
int schemaNameLength = schemaName.getBytes().length;
int tableNameLength = keyLength - schemaNameLength - 1 - tenantIdLength - 1;
byte[] tableNameBytes = new byte[tableNameLength];
System.arraycopy(keyBuffer, keyOffset + schemaNameLength + 1 + tenantIdLength + 1,
tableNameBytes, 0, tableNameLength);
PName tableName = PNameFactory.newName(tableNameBytes);
int offset = tenantIdLength + schemaNameLength + tableNameLength + 3;
// This will prevent the client from continually looking for the current
// table when we know that there will never be one since we disallow updates
// unless the table is the latest
long timeStamp = keyValue.getTimestamp();
PTableImpl.Builder builder = null;
if (oldTable != null) {
builder = PTableImpl.builderFromExisting(oldTable);
List<PColumn> columns = oldTable.getColumns();
if (oldTable.getBucketNum() != null && oldTable.getBucketNum() > 0) {
//if it's salted, skip the salt column -- it will get added back during
//the build process
columns = columns.stream().skip(1).collect(Collectors.toList());
}
builder.setColumns(columns);
} else {
builder = new PTableImpl.Builder();
}
builder.setTenantId(tenantId);
builder.setSchemaName(schemaName);
builder.setTableName(tableName);
int i = 0;
int j = 0;
while (i < tableCellList.size() && j < TABLE_KV_COLUMNS.size()) {
Cell kv = tableCellList.get(i);
Cell searchKv = TABLE_KV_COLUMNS.get(j);
int cmp =
Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
kv.getQualifierLength(), searchKv.getQualifierArray(),
searchKv.getQualifierOffset(), searchKv.getQualifierLength());
if (cmp == 0) {
timeStamp = Math.max(timeStamp, kv.getTimestamp()); // Find max timestamp of table
// header row
tableKeyValues[j++] = kv;
i++;
} else if (cmp > 0) {
timeStamp = Math.max(timeStamp, kv.getTimestamp());
tableKeyValues[j++] = null;
} else {
i++; // shouldn't happen - means unexpected KV in system table header row
}
}
// TABLE_TYPE, TABLE_SEQ_NUM and COLUMN_COUNT are required.
if (tableKeyValues[TABLE_TYPE_INDEX] == null || tableKeyValues[TABLE_SEQ_NUM_INDEX] == null
|| tableKeyValues[COLUMN_COUNT_INDEX] == null) {
// since we allow SYSTEM.CATALOG to split in certain cases there might be child links or
// other metadata rows that are invalid and should be ignored
Cell cell = tableCellList.get(0);
LOGGER.error("Found invalid metadata rows for rowkey " +
Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
return null;
}
Cell tableTypeKv = tableKeyValues[TABLE_TYPE_INDEX];
PTableType tableType =
PTableType
.fromSerializedValue(tableTypeKv.getValueArray()[tableTypeKv.getValueOffset()]);
builder.setType(tableType);
Cell tableSeqNumKv = tableKeyValues[TABLE_SEQ_NUM_INDEX];
long tableSeqNum =
PLong.INSTANCE.getCodec().decodeLong(tableSeqNumKv.getValueArray(),
tableSeqNumKv.getValueOffset(), SortOrder.getDefault());
builder.setSequenceNumber(tableSeqNum);
Cell columnCountKv = tableKeyValues[COLUMN_COUNT_INDEX];
int columnCount =
PInteger.INSTANCE.getCodec().decodeInt(columnCountKv.getValueArray(),
columnCountKv.getValueOffset(), SortOrder.getDefault());
Cell pkNameKv = tableKeyValues[PK_NAME_INDEX];
PName pkName =
pkNameKv != null ? newPName(pkNameKv.getValueArray(), pkNameKv.getValueOffset(),
pkNameKv.getValueLength()) : null;
builder.setPkName(pkName != null ? pkName : oldTable != null ? oldTable.getPKName() : null);
Cell saltBucketNumKv = tableKeyValues[SALT_BUCKETS_INDEX];
Integer saltBucketNum =
saltBucketNumKv != null ? (Integer) PInteger.INSTANCE.getCodec().decodeInt(
saltBucketNumKv.getValueArray(), saltBucketNumKv.getValueOffset(), SortOrder.getDefault()) : null;
if (saltBucketNum != null && saltBucketNum.intValue() == 0) {
saltBucketNum = null; // Zero salt buckets means not salted
}
builder.setBucketNum(saltBucketNum != null ? saltBucketNum : oldTable != null ? oldTable.getBucketNum() : null);
//data table name is used to find the parent table for indexes later
Cell dataTableNameKv = tableKeyValues[DATA_TABLE_NAME_INDEX];
PName dataTableName =
dataTableNameKv != null ? newPName(dataTableNameKv.getValueArray(),
dataTableNameKv.getValueOffset(), dataTableNameKv.getValueLength()) : null;
Cell physicalTableNameKv = tableKeyValues[PHYSICAL_TABLE_NAME_INDEX];
PName physicalTableName =
physicalTableNameKv != null ? newPName(physicalTableNameKv.getValueArray(),
physicalTableNameKv.getValueOffset(), physicalTableNameKv.getValueLength()) : null;
builder.setPhysicalTableName(physicalTableName != null ? physicalTableName : oldTable != null ? oldTable.getPhysicalName(true) : null);
Cell indexStateKv = tableKeyValues[INDEX_STATE_INDEX];
PIndexState indexState =
indexStateKv == null ? null : PIndexState.fromSerializedValue(indexStateKv
.getValueArray()[indexStateKv.getValueOffset()]);
builder.setState(indexState != null ? indexState : oldTable != null ? oldTable.getIndexState() : null);
Cell immutableRowsKv = tableKeyValues[IMMUTABLE_ROWS_INDEX];
boolean isImmutableRows = immutableRowsKv != null && (Boolean) PBoolean.INSTANCE.toObject(
immutableRowsKv.getValueArray(), immutableRowsKv.getValueOffset(),
immutableRowsKv.getValueLength());
builder.setImmutableRows(immutableRowsKv != null ? isImmutableRows :
oldTable != null && oldTable.isImmutableRows());
Cell defaultFamilyNameKv = tableKeyValues[DEFAULT_COLUMN_FAMILY_INDEX];
PName defaultFamilyName = defaultFamilyNameKv != null ? newPName(defaultFamilyNameKv.getValueArray(), defaultFamilyNameKv.getValueOffset(), defaultFamilyNameKv.getValueLength()) : null;
builder.setDefaultFamilyName(defaultFamilyName != null ? defaultFamilyName : oldTable != null ? oldTable.getDefaultFamilyName() : null);
Cell viewStatementKv = tableKeyValues[VIEW_STATEMENT_INDEX];
String viewStatement = viewStatementKv != null ? (String) PVarchar.INSTANCE.toObject(viewStatementKv.getValueArray(), viewStatementKv.getValueOffset(),
viewStatementKv.getValueLength()) : null;
builder.setViewStatement(viewStatement != null ? viewStatement : oldTable != null ? oldTable.getViewStatement() : null);
Cell disableWALKv = tableKeyValues[DISABLE_WAL_INDEX];
boolean disableWAL = disableWALKv == null ? PTable.DEFAULT_DISABLE_WAL : Boolean.TRUE.equals(
PBoolean.INSTANCE.toObject(disableWALKv.getValueArray(), disableWALKv.getValueOffset(), disableWALKv.getValueLength()));
builder.setDisableWAL(disableWALKv != null ? disableWAL :
oldTable != null && oldTable.isWALDisabled());
Cell multiTenantKv = tableKeyValues[MULTI_TENANT_INDEX];
boolean multiTenant = multiTenantKv != null && Boolean.TRUE.equals(
PBoolean.INSTANCE.toObject(multiTenantKv.getValueArray(),
multiTenantKv.getValueOffset(), multiTenantKv.getValueLength()));
builder.setMultiTenant(multiTenantKv != null ? multiTenant :
oldTable != null && oldTable.isMultiTenant());
Cell storeNullsKv = tableKeyValues[STORE_NULLS_INDEX];
boolean storeNulls = storeNullsKv != null && Boolean.TRUE.equals(
PBoolean.INSTANCE.toObject(storeNullsKv.getValueArray(), storeNullsKv.getValueOffset(),
storeNullsKv.getValueLength()));
builder.setStoreNulls(storeNullsKv != null ? storeNulls :
oldTable != null && oldTable.getStoreNulls());
Cell transactionalKv = tableKeyValues[TRANSACTIONAL_INDEX];
Cell transactionProviderKv = tableKeyValues[TRANSACTION_PROVIDER_INDEX];
TransactionFactory.Provider transactionProvider = null;
if (transactionProviderKv == null) {
if (transactionalKv != null && Boolean.TRUE.equals(
PBoolean.INSTANCE.toObject(
transactionalKv.getValueArray(),
transactionalKv.getValueOffset(),
transactionalKv.getValueLength()))) {
// For backward compat, prior to client setting TRANSACTION_PROVIDER
transactionProvider = TransactionFactory.Provider.NOTAVAILABLE;
}
} else {
transactionProvider = TransactionFactory.Provider.fromCode(
PTinyint.INSTANCE.getCodec().decodeByte(
transactionProviderKv.getValueArray(),
transactionProviderKv.getValueOffset(),
SortOrder.getDefault()));
}
builder.setTransactionProvider(transactionProviderKv != null || transactionalKv != null
? transactionProvider : oldTable != null ? oldTable.getTransactionProvider() : null);
Cell viewTypeKv = tableKeyValues[VIEW_TYPE_INDEX];
ViewType viewType = viewTypeKv == null ? null : ViewType.fromSerializedValue(viewTypeKv.getValueArray()[viewTypeKv.getValueOffset()]);
builder.setViewType(viewType != null ? viewType : oldTable != null ? oldTable.getViewType() : null);
PDataType viewIndexIdType = oldTable != null ? oldTable.getviewIndexIdType() :
getViewIndexIdType(tableKeyValues);
builder.setViewIndexIdType(viewIndexIdType);
Long viewIndexId = getViewIndexId(tableKeyValues, viewIndexIdType);
builder.setViewIndexId(viewIndexId != null ? viewIndexId : oldTable != null ? oldTable.getViewIndexId() : null);
Cell indexTypeKv = tableKeyValues[INDEX_TYPE_INDEX];
IndexType indexType = indexTypeKv == null ? null : IndexType.fromSerializedValue(indexTypeKv.getValueArray()[indexTypeKv.getValueOffset()]);
builder.setIndexType(indexType != null ? indexType : oldTable != null ? oldTable.getIndexType() : null);
Cell baseColumnCountKv = tableKeyValues[BASE_COLUMN_COUNT_INDEX];
int baseColumnCount = baseColumnCountKv == null ? 0 : PInteger.INSTANCE.getCodec().decodeInt(baseColumnCountKv.getValueArray(),
baseColumnCountKv.getValueOffset(), SortOrder.getDefault());
builder.setBaseColumnCount(baseColumnCountKv != null ? baseColumnCount : oldTable != null ? oldTable.getBaseColumnCount() : 0);
Cell rowKeyOrderOptimizableKv = tableKeyValues[ROW_KEY_ORDER_OPTIMIZABLE_INDEX];
boolean rowKeyOrderOptimizable = rowKeyOrderOptimizableKv != null && Boolean.TRUE.equals(
PBoolean.INSTANCE.toObject(rowKeyOrderOptimizableKv.getValueArray(),
rowKeyOrderOptimizableKv.getValueOffset(),
rowKeyOrderOptimizableKv.getValueLength()));
builder.setRowKeyOrderOptimizable(rowKeyOrderOptimizableKv != null ? rowKeyOrderOptimizable :
oldTable != null && oldTable.rowKeyOrderOptimizable());
Cell updateCacheFrequencyKv = tableKeyValues[UPDATE_CACHE_FREQUENCY_INDEX];
long updateCacheFrequency = updateCacheFrequencyKv == null ? 0 :
PLong.INSTANCE.getCodec().decodeLong(updateCacheFrequencyKv.getValueArray(),
updateCacheFrequencyKv.getValueOffset(), SortOrder.getDefault());
builder.setUpdateCacheFrequency(updateCacheFrequencyKv != null ? updateCacheFrequency : oldTable != null ? oldTable.getUpdateCacheFrequency() : 0);
// Check the cell tag to see whether the view has modified this property
final byte[] tagUpdateCacheFreq = (updateCacheFrequencyKv == null) ?
HConstants.EMPTY_BYTE_ARRAY :
TagUtil.concatTags(HConstants.EMPTY_BYTE_ARRAY, updateCacheFrequencyKv);
boolean viewModifiedUpdateCacheFrequency = (PTableType.VIEW.equals(tableType)) &&
Bytes.contains(tagUpdateCacheFreq, MetaDataEndpointImplConstants.VIEW_MODIFIED_PROPERTY_BYTES);
builder.setViewModifiedUpdateCacheFrequency(!Bytes.equals(tagUpdateCacheFreq,
HConstants.EMPTY_BYTE_ARRAY) ? viewModifiedUpdateCacheFrequency :
oldTable != null && oldTable.hasViewModifiedUpdateCacheFrequency());
Cell indexDisableTimestampKv = tableKeyValues[INDEX_DISABLE_TIMESTAMP];
long indexDisableTimestamp = indexDisableTimestampKv == null ? 0L : PLong.INSTANCE.getCodec().decodeLong(indexDisableTimestampKv.getValueArray(),
indexDisableTimestampKv.getValueOffset(), SortOrder.getDefault());
builder.setIndexDisableTimestamp(indexDisableTimestampKv != null ?
indexDisableTimestamp : oldTable != null ? oldTable.getIndexDisableTimestamp() : 0L);
Cell isNamespaceMappedKv = tableKeyValues[IS_NAMESPACE_MAPPED_INDEX];
boolean isNamespaceMapped = isNamespaceMappedKv != null && Boolean.TRUE.equals(
PBoolean.INSTANCE.toObject(isNamespaceMappedKv.getValueArray(),
isNamespaceMappedKv.getValueOffset(), isNamespaceMappedKv.getValueLength()));
builder.setNamespaceMapped(isNamespaceMappedKv != null ? isNamespaceMapped :
oldTable != null && oldTable.isNamespaceMapped());
Cell autoPartitionSeqKv = tableKeyValues[AUTO_PARTITION_SEQ_INDEX];
String autoPartitionSeq = autoPartitionSeqKv != null ? (String) PVarchar.INSTANCE.toObject(autoPartitionSeqKv.getValueArray(), autoPartitionSeqKv.getValueOffset(),
autoPartitionSeqKv.getValueLength()) : null;
builder.setAutoPartitionSeqName(autoPartitionSeq != null
? autoPartitionSeq : oldTable != null ? oldTable.getAutoPartitionSeqName() : null);
Cell isAppendOnlySchemaKv = tableKeyValues[APPEND_ONLY_SCHEMA_INDEX];
boolean isAppendOnlySchema = isAppendOnlySchemaKv != null && Boolean.TRUE.equals(
PBoolean.INSTANCE.toObject(isAppendOnlySchemaKv.getValueArray(),
isAppendOnlySchemaKv.getValueOffset(), isAppendOnlySchemaKv.getValueLength()));
builder.setAppendOnlySchema(isAppendOnlySchemaKv != null ? isAppendOnlySchema :
oldTable != null && oldTable.isAppendOnlySchema());
Cell storageSchemeKv = tableKeyValues[STORAGE_SCHEME_INDEX];
//TODO: change this once we start having other values for storage schemes
ImmutableStorageScheme storageScheme = storageSchemeKv == null ? ImmutableStorageScheme.ONE_CELL_PER_COLUMN : ImmutableStorageScheme
.fromSerializedValue((byte) PTinyint.INSTANCE.toObject(storageSchemeKv.getValueArray(),
storageSchemeKv.getValueOffset(), storageSchemeKv.getValueLength()));
builder.setImmutableStorageScheme(storageSchemeKv != null ? storageScheme :
oldTable != null ? oldTable.getImmutableStorageScheme() : ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
Cell encodingSchemeKv = tableKeyValues[QUALIFIER_ENCODING_SCHEME_INDEX];
QualifierEncodingScheme encodingScheme = encodingSchemeKv == null ? QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : QualifierEncodingScheme
.fromSerializedValue((byte) PTinyint.INSTANCE.toObject(encodingSchemeKv.getValueArray(),
encodingSchemeKv.getValueOffset(), encodingSchemeKv.getValueLength()));
builder.setQualifierEncodingScheme(encodingSchemeKv != null ? encodingScheme :
oldTable != null ? oldTable.getEncodingScheme() : QualifierEncodingScheme.NON_ENCODED_QUALIFIERS);
Cell useStatsForParallelizationKv = tableKeyValues[USE_STATS_FOR_PARALLELIZATION_INDEX];
Boolean useStatsForParallelization = useStatsForParallelizationKv == null ? null :
Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(useStatsForParallelizationKv.getValueArray(), useStatsForParallelizationKv.getValueOffset(), useStatsForParallelizationKv.getValueLength()));
builder.setUseStatsForParallelization(useStatsForParallelization != null ?
useStatsForParallelization : oldTable != null ? oldTable.useStatsForParallelization() : null);
Cell lastDDLTimestampKv = tableKeyValues[LAST_DDL_TIMESTAMP_INDEX];
Long lastDDLTimestamp = lastDDLTimestampKv == null ?
null : PLong.INSTANCE.getCodec().decodeLong(lastDDLTimestampKv.getValueArray(),
lastDDLTimestampKv.getValueOffset(), SortOrder.getDefault());
builder.setLastDDLTimestamp(lastDDLTimestampKv != null ? lastDDLTimestamp :
oldTable != null ? oldTable.getLastDDLTimestamp() : timeStamp);
Cell changeDetectionEnabledKv = tableKeyValues[CHANGE_DETECTION_ENABLED_INDEX];
boolean isChangeDetectionEnabled = changeDetectionEnabledKv != null
&& Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(changeDetectionEnabledKv.getValueArray(),
changeDetectionEnabledKv.getValueOffset(),
changeDetectionEnabledKv.getValueLength()));
builder.setIsChangeDetectionEnabled(changeDetectionEnabledKv != null ?
isChangeDetectionEnabled : oldTable != null && oldTable.isChangeDetectionEnabled());
Cell schemaVersionKv = tableKeyValues[SCHEMA_VERSION_INDEX];
String schemaVersion = schemaVersionKv != null ? (String) PVarchar.INSTANCE.toObject(
schemaVersionKv.getValueArray(), schemaVersionKv.getValueOffset(), schemaVersionKv.getValueLength())
: null;
builder.setSchemaVersion(schemaVersion != null ?
schemaVersion : oldTable != null ? oldTable.getSchemaVersion() : null);
Cell externalSchemaIdKv = tableKeyValues[EXTERNAL_SCHEMA_ID_INDEX];
String externalSchemaId = externalSchemaIdKv != null ?
(String) PVarchar.INSTANCE.toObject(externalSchemaIdKv.getValueArray(),
externalSchemaIdKv.getValueOffset(), externalSchemaIdKv.getValueLength())
: null;
builder.setExternalSchemaId(externalSchemaId != null ? externalSchemaId :
oldTable != null ? oldTable.getExternalSchemaId() : null);
Cell streamingTopicNameKv = tableKeyValues[STREAMING_TOPIC_NAME_INDEX];
String streamingTopicName = streamingTopicNameKv != null ?
(String) PVarchar.INSTANCE.toObject(streamingTopicNameKv.getValueArray(),
streamingTopicNameKv.getValueOffset(), streamingTopicNameKv.getValueLength())
: null;
builder.setStreamingTopicName(streamingTopicName != null ? streamingTopicName :
oldTable != null ? oldTable.getStreamingTopicName() : null);
Cell includeSpecKv = tableKeyValues[CDC_INCLUDE_INDEX];
String includeSpec = includeSpecKv != null ?
(String) PVarchar.INSTANCE.toObject(includeSpecKv.getValueArray(),
includeSpecKv.getValueOffset(), includeSpecKv.getValueLength())
: null;
builder.setCDCIncludeScopes(includeSpec != null ?
CDCUtil.makeChangeScopeEnumsFromString(includeSpec) :
oldTable != null ? oldTable.getCDCIncludeScopes() : null);
Cell indexWhereKv = tableKeyValues[INDEX_WHERE_INDEX];
String indexWhere = indexWhereKv != null
? (String) PVarchar.INSTANCE.toObject(indexWhereKv.getValueArray(),
indexWhereKv.getValueOffset(), indexWhereKv.getValueLength())
: null;
builder.setIndexWhere(indexWhere != null ? indexWhere
: oldTable != null ? oldTable.getIndexWhere() : null);
Cell ttlKv = tableKeyValues[TTL_INDEX];
TTLExpression ttl = TTL_EXPRESSION_NOT_DEFINED;
if (ttlKv != null) {
String ttlStr = (String) PVarchar.INSTANCE.toObject(
ttlKv.getValueArray(),
ttlKv.getValueOffset(),
ttlKv.getValueLength());
ttl = TTLExpressionFactory.create(ttlStr);
}
ttl = ttlKv != null ? ttl : oldTable != null
? oldTable.getTTLExpression() : TTL_EXPRESSION_NOT_DEFINED;
if (tableType == VIEW && viewType != MAPPED && ttl.equals(TTL_EXPRESSION_NOT_DEFINED)) {
//Scan SysCat to get TTL from Parent View/Table
byte[] viewKey = SchemaUtil.getTableKey(tenantId == null ? null : tenantId.getBytes(),
schemaName == null ? null : schemaName.getBytes(), tableNameBytes);
ttl = getTTLFromHierarchy(viewKey, clientTimeStamp, false);
// TODO: Need to Update Cache for Alter Commands, can use PHOENIX-6883.
}
Cell rowKeyMatcherKv = tableKeyValues[ROW_KEY_MATCHER_INDEX];
byte[] rowKeyMatcher = rowKeyMatcherKv != null
? CellUtil.cloneValue(rowKeyMatcherKv)
: HConstants.EMPTY_BYTE_ARRAY;
builder.setRowKeyMatcher(rowKeyMatcher != null ? rowKeyMatcher
: oldTable != null ? oldTable.getRowKeyMatcher() : HConstants.EMPTY_BYTE_ARRAY);
// Check the cell tag to see whether the view has modified this property
final byte[] tagUseStatsForParallelization = (useStatsForParallelizationKv == null) ?
HConstants.EMPTY_BYTE_ARRAY :
TagUtil.concatTags(HConstants.EMPTY_BYTE_ARRAY, useStatsForParallelizationKv);
boolean viewModifiedUseStatsForParallelization = (PTableType.VIEW.equals(tableType)) &&
Bytes.contains(tagUseStatsForParallelization, MetaDataEndpointImplConstants.VIEW_MODIFIED_PROPERTY_BYTES);
builder.setViewModifiedUseStatsForParallelization(viewModifiedUseStatsForParallelization ||
(oldTable != null && oldTable.hasViewModifiedUseStatsForParallelization()));
boolean setPhysicalName = false;
List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
List<PTable> indexes = Lists.newArrayList();
List<PName> physicalTables = Lists.newArrayList();
PName parentTableName = tableType == INDEX || tableType == CDC ? dataTableName : null;
PName parentSchemaName = tableType == INDEX || tableType == CDC ? schemaName : null;
PName parentLogicalName = null;
EncodedCQCounter cqCounter = null;
if (oldTable != null) {
cqCounter = oldTable.getEncodedCQCounter();
} else {
cqCounter = (!EncodedColumnsUtil.usesEncodedColumnNames(encodingScheme) || tableType == PTableType.VIEW) ?
PTable.EncodedCQCounter.NULL_COUNTER :
new EncodedCQCounter();
}
if (timeStamp == HConstants.LATEST_TIMESTAMP) {
timeStamp = lastDDLTimestamp != null ? lastDDLTimestamp : clientTimeStamp;
}
builder.setTimeStamp(timeStamp);
PTable transformingNewTable = null;
boolean isRegularView = (tableType == PTableType.VIEW && viewType != MAPPED);
boolean isThisAViewIndex = false;
for (List<Cell> columnCellList : allColumnCellList) {
Cell colKv = columnCellList.get(LINK_TYPE_INDEX);
int colKeyLength = colKv.getRowLength();
PName colName = newPName(colKv.getRowArray(), colKv.getRowOffset() + offset, colKeyLength - offset);
if (colName == null) {
continue;
}
int colKeyOffset = offset + colName.getBytes().length + 1;
PName famName = newPName(colKv.getRowArray(), colKv.getRowOffset() + colKeyOffset, colKeyLength - colKeyOffset);
if (isQualifierCounterKV(colKv)) {
if (famName != null) {
Integer value = PInteger.INSTANCE.getCodec().decodeInt(colKv.getValueArray(), colKv.getValueOffset(), SortOrder.ASC);
cqCounter.setValue(famName.getString(), value);
}
} else if (Bytes.compareTo(LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length, colKv.getQualifierArray(), colKv.getQualifierOffset(), colKv.getQualifierLength()) == 0) {
LinkType linkType = LinkType.fromSerializedValue(colKv.getValueArray()[colKv.getValueOffset()]);
if (linkType == LinkType.INDEX_TABLE) {
addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes, clientVersion);
} else if (linkType == PHYSICAL_TABLE) {
// famName contains the logical name of the parent table. We need to get the actual physical name of the table
PTable parentTable = null;
// call getTable() on famName only if it does not start with _IDX_.
// Table name starting with _IDX_ always must refer to HBase table that is
// shared by all view indexes on the given table/view hierarchy.
// _IDX_ is HBase table that does not have corresponding PTable representation
// in Phoenix, hence there is no point of calling getTable().
if (!famName.getString().startsWith(MetaDataUtil.VIEW_INDEX_TABLE_PREFIX)
&& indexType != IndexType.LOCAL) {
try {
parentTable = doGetTable(null,
SchemaUtil.getSchemaNameFromFullName(famName.getBytes())
.getBytes(StandardCharsets.UTF_8),
SchemaUtil.getTableNameFromFullName(famName.getBytes())
.getBytes(StandardCharsets.UTF_8), clientTimeStamp,
clientVersion);
} catch (SQLException e) {
if (e.getErrorCode()
!= SQLExceptionCode.GET_TABLE_ERROR.getErrorCode()) {
LOGGER.error(
"Error while retrieving getTable for PHYSICAL_TABLE link to {}",
famName, e);
throw e;
}
}
if (isSystemCatalogSplittable
&& (parentTable == null || isTableDeleted(parentTable))) {
// parentTable is neither in the cache nor in the local region. Since
// famName is only logical name, we need to find the physical table.
// Hence, it is recommended to scan SYSTEM.CATALOG table again using
// separate CQSI connection as SYSTEM.CATALOG is splittable so the
// PTable with famName might be available on different region.
try (PhoenixConnection connection = getServerConnectionForMetaData(
env.getConfiguration()).unwrap(PhoenixConnection.class)) {
parentTable = connection.getTableNoCache(famName.getString());
} catch (TableNotFoundException e) {
// It is ok to swallow this exception since this could be a view index and _IDX_ table is not there.
}
}
}
if (parentTable == null || isTableDeleted(parentTable)) {
if (indexType == IndexType.LOCAL) {
PName tablePhysicalName = getPhysicalTableName(
env.getRegion(),null,
SchemaUtil.getSchemaNameFromFullName(
famName.getBytes()).getBytes(StandardCharsets.UTF_8),
SchemaUtil.getTableNameFromFullName(
famName.getBytes()).getBytes(StandardCharsets.UTF_8),
clientTimeStamp);
if (tablePhysicalName == null) {
physicalTables.add(famName);
setPhysicalName = true;
} else {
physicalTables.add(SchemaUtil.getPhysicalHBaseTableName(schemaName, tablePhysicalName, isNamespaceMapped));
setPhysicalName = true;
}
} else {
physicalTables.add(famName);
setPhysicalName = true;
}
// If this is a view index, then one of the link is IDX_VW -> _IDX_ PhysicalTable link. Since famName is _IDX_ and we can't get this table hence it is null, we need to use actual view name
parentLogicalName = (tableType == INDEX ? SchemaUtil.getTableName(parentSchemaName, parentTableName) : famName);
} else {
String parentPhysicalTableName = parentTable.getPhysicalName().getString();
physicalTables.add(PNameFactory.newName(parentPhysicalTableName));
setPhysicalName = true;
parentLogicalName = SchemaUtil.getTableName(parentTable.getSchemaName(), parentTable.getTableName());
}
} else if (linkType == PARENT_TABLE) {
parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(famName.getBytes()));
parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName(famName.getBytes()));
} else if (linkType == LinkType.EXCLUDED_COLUMN) {
// add the excludedColumn
addExcludedColumnToTable(columns, colName, famName, colKv.getTimestamp());
} else if (linkType == LinkType.TRANSFORMING_NEW_TABLE) {
transformingNewTable = doGetTable((tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes())
, SchemaUtil.getSchemaNameFromFullName(famName.getBytes()).getBytes(), SchemaUtil.getTableNameFromFullName(famName.getBytes()).getBytes(), clientTimeStamp, null, clientVersion);
if (transformingNewTable == null) {
// It could be global
transformingNewTable = doGetTable(ByteUtil.EMPTY_BYTE_ARRAY
, SchemaUtil.getSchemaNameFromFullName(famName.getBytes()).getBytes(), SchemaUtil.getTableNameFromFullName(famName.getBytes()).getBytes(), clientTimeStamp, null, clientVersion);
if (transformingNewTable == null) {
ClientUtil.throwIOException("Transforming new table not found", new TableNotFoundException(schemaName.getString(), famName.getString()));
}
}
} else if (linkType == VIEW_INDEX_PARENT_TABLE) {
byte[] viewKey = getTableKey(tenantId == null ? null : tenantId.getBytes(),
parentSchemaName == null ? null : parentSchemaName.getBytes(),
parentTableName.getBytes());
//parentViewType should not be Mapped
ttl = getTTLFromHierarchy(viewKey, clientTimeStamp, true);
isThisAViewIndex = true;
}
} else {
long columnTimestamp =
columnCellList.get(0).getTimestamp() != HConstants.LATEST_TIMESTAMP ?
columnCellList.get(0).getTimestamp() : timeStamp;
boolean isSalted = saltBucketNum != null
|| (oldTable != null &&
oldTable.getBucketNum() != null && oldTable.getBucketNum() > 0);
addColumnToTable(columnCellList, colName, famName, colKeyValues, columns,
isSalted, baseColumnCount, isRegularView, columnTimestamp);
}
}
// Ignoring meta indexes when looking for TTL on parent tables.
// Due to failures in Namespace related ITs when isNamespaceMappingEnabled is enabled.
boolean isMetaIndex = (QueryConstants.SYSTEM_SCHEMA_NAME.equals(schemaName.getString())
&& (tableType == INDEX));
if (tableType == INDEX && !isThisAViewIndex && ttl.equals(TTL_EXPRESSION_NOT_DEFINED) && !isMetaIndex) {
//If this is an index on Table get TTL from Table
byte[] tableKey = getTableKey(tenantId == null ? null : tenantId.getBytes(),
parentSchemaName == null ? null : parentSchemaName.getBytes(),
parentTableName.getBytes());
ttl = getTTLForTable(tableKey, clientTimeStamp);
}
if (tableType == INDEX
&& CDCUtil.isCDCIndex(tableName.getString())
&& !ttl.equals(TTL_EXPRESSION_NOT_DEFINED)) {
ttl = TTL_EXPRESSION_FOREVER;
}
builder.setTTL(ttl);
builder.setEncodedCQCounter(cqCounter);
builder.setIndexes(indexes != null ? indexes : oldTable != null
? oldTable.getIndexes() : Collections.<PTable>emptyList());
if (physicalTables == null || physicalTables.size() == 0) {
builder.setPhysicalNames(oldTable != null ? oldTable.getPhysicalNames()
: ImmutableList.<PName>of());
} else {
builder.setPhysicalNames(ImmutableList.copyOf(physicalTables));
}
if (!setPhysicalName && oldTable != null) {
builder.setPhysicalTableName(oldTable.getPhysicalName(true));
}
builder.setTransformingNewTable(transformingNewTable);
builder.setExcludedColumns(ImmutableList.<PColumn>of());
builder.setBaseTableLogicalName(parentLogicalName != null ?
parentLogicalName : oldTable != null ? oldTable.getBaseTableLogicalName() : null);
builder.setParentTableName(parentTableName != null ?
parentTableName : oldTable != null ? oldTable.getParentTableName() : null);
builder.setParentSchemaName(parentSchemaName != null ? parentSchemaName :
oldTable != null ? oldTable.getParentSchemaName() : null);
builder.addOrSetColumns(columns);
// Avoid querying the stats table because we're holding the rowLock here. Issuing an RPC to a remote
// server while holding this lock is a bad idea and likely to cause contention.
return builder.build();
}