Skip to content

Commit

Permalink
Add a builder for HiveTableHandle
Browse files Browse the repository at this point in the history
  • Loading branch information
assaf2 authored and raunaqmorarka committed Dec 4, 2022
1 parent 56936ba commit 3350aa8
Show file tree
Hide file tree
Showing 11 changed files with 320 additions and 252 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -482,13 +482,14 @@ public HiveTableHandle getTableHandle(ConnectorSession session, SchemaTableName

verifyOnline(tableName, Optional.empty(), getProtectMode(table), table.getParameters());

return new HiveTableHandle(
tableName.getSchemaName(),
tableName.getTableName(),
table.getParameters(),
getPartitionKeyColumnHandles(table, typeManager),
getRegularColumnHandles(table, typeManager, getTimestampPrecision(session)),
getHiveBucketHandle(session, table, typeManager));
return HiveTableHandle.builder()
.withSchemaName(tableName.getSchemaName())
.withTableName(tableName.getTableName())
.withTableParameters(Optional.of(table.getParameters()))
.withPartitionColumns(getPartitionKeyColumnHandles(table, typeManager))
.withDataColumns(getRegularColumnHandles(table, typeManager, getTimestampPrecision(session)))
.withBucketHandle(getHiveBucketHandle(session, table, typeManager))
.build();
}

@Override
Expand All @@ -514,7 +515,9 @@ public ConnectorAnalyzeMetadata getStatisticsCollectionMetadata(ConnectorSession
}
}

handle = handle.withAnalyzePartitionValues(list);
handle = HiveTableHandle.buildFrom(handle)
.withAnalyzePartitionValues(Optional.of(list))
.build();
HivePartitionResult partitions = partitionManager.getPartitions(handle, list);
handle = partitionManager.applyPartitionResult(handle, partitions, alwaysTrue());
}
Expand Down Expand Up @@ -1886,7 +1889,9 @@ public ConnectorTableHandle beginUpdate(ConnectorSession session, ConnectorTable

HiveUpdateProcessor updateProcessor = new HiveUpdateProcessor(allDataColumns, hiveUpdatedColumns);
AcidTransaction transaction = metastore.beginUpdate(session, table, updateProcessor);
HiveTableHandle updateHandle = hiveTableHandle.withTransaction(transaction);
HiveTableHandle updateHandle = HiveTableHandle.buildFrom(hiveTableHandle)
.withTransaction(transaction)
.build();

WriteInfo writeInfo = locationService.getQueryWriteInfo(locationHandle);
metastore.declareIntentionToWrite(session, writeInfo.getWriteMode(), writeInfo.getWritePath(), tableName);
Expand Down Expand Up @@ -1947,7 +1952,11 @@ public ConnectorMergeTableHandle beginMerge(ConnectorSession session, ConnectorT
}

HiveInsertTableHandle insertHandle = beginInsertOrMerge(session, tableHandle, retryMode, "Merging into", true);
return new HiveMergeTableHandle(hiveTableHandle.withTransaction(insertHandle.getTransaction()), insertHandle);
return new HiveMergeTableHandle(
HiveTableHandle.buildFrom(hiveTableHandle)
.withTransaction(insertHandle.getTransaction())
.build(),
insertHandle);
}

@Override
Expand Down Expand Up @@ -2420,9 +2429,10 @@ private BeginTableExecuteResult<ConnectorTableExecuteHandle, ConnectorTableHandl
return new BeginTableExecuteResult<>(
hiveExecuteHandle
.withWriteDeclarationId(writeDeclarationId),
hiveSourceTableHandle
HiveTableHandle.buildFrom(hiveSourceTableHandle)
.withMaxScannedFileSize(hiveExecuteHandle.getMaxScannedFileSize())
.withRecordScannedFiles(true));
.withRecordScannedFiles(true)
.build());
}

@Override
Expand Down Expand Up @@ -2770,7 +2780,9 @@ public ConnectorTableHandle beginDelete(ConnectorSession session, ConnectorTable
WriteInfo writeInfo = locationService.getQueryWriteInfo(locationHandle);
metastore.declareIntentionToWrite(session, writeInfo.getWriteMode(), writeInfo.getWritePath(), handle.getSchemaTableName());

return handle.withTransaction(transaction);
return HiveTableHandle.buildFrom(handle)
.withTransaction(transaction)
.build();
}

@Override
Expand Down Expand Up @@ -3042,7 +3054,9 @@ public Optional<ProjectionApplicationResult<ConnectorTableHandle>> applyProjecti
((HiveColumnHandle) assignment.getValue()).getType()))
.collect(toImmutableList());
return Optional.of(new ProjectionApplicationResult<>(
hiveTableHandle.withProjectedColumns(projectedColumns),
HiveTableHandle.buildFrom(hiveTableHandle)
.withProjectedColumns(projectedColumns)
.build(),
projections,
assignmentsList,
false));
Expand Down Expand Up @@ -3089,7 +3103,9 @@ public Optional<ProjectionApplicationResult<ConnectorTableHandle>> applyProjecti

List<Assignment> outputAssignments = ImmutableList.copyOf(newAssignments.values());
return Optional.of(new ProjectionApplicationResult<>(
hiveTableHandle.withProjectedColumns(projectedColumnsBuilder.build()),
HiveTableHandle.buildFrom(hiveTableHandle)
.withProjectedColumns(projectedColumnsBuilder.build())
.build(),
newProjections,
outputAssignments,
false));
Expand Down Expand Up @@ -3213,29 +3229,16 @@ public ConnectorTableHandle makeCompatiblePartitioning(ConnectorSession session,
largerBucketCount % smallerBucketCount == 0 && Integer.bitCount(largerBucketCount / smallerBucketCount) == 1,
"The requested partitioning is not a valid alternative for the table layout");

return new HiveTableHandle(
hiveTable.getSchemaName(),
hiveTable.getTableName(),
hiveTable.getTableParameters(),
hiveTable.getPartitionColumns(),
hiveTable.getDataColumns(),
hiveTable.getPartitionNames(),
hiveTable.getPartitions(),
hiveTable.getCompactEffectivePredicate(),
hiveTable.getEnforcedConstraint(),
Optional.of(new HiveBucketHandle(
return HiveTableHandle.buildFrom(hiveTable)
.withBucketHandle(Optional.of(new HiveBucketHandle(
bucketHandle.getColumns(),
bucketHandle.getBucketingVersion(),
bucketHandle.getTableBucketCount(),
hivePartitioningHandle.getBucketCount(),
bucketHandle.getSortedBy())),
hiveTable.getBucketFilter(),
hiveTable.getAnalyzePartitionValues(),
ImmutableSet.of(),
ImmutableSet.of(), // Projected columns is used only during optimization phase of planning
hiveTable.getTransaction(),
hiveTable.isRecordScannedFiles(),
hiveTable.getMaxScannedFileSize());
bucketHandle.getSortedBy())))
.withConstraintColumns(ImmutableSet.of())
.withProjectedColumns(ImmutableSet.of()) // Projected columns is used only during optimization phase of planning
.build();
}

@VisibleForTesting
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,24 +168,16 @@ public HiveTableHandle applyPartitionResult(HiveTableHandle handle, HivePartitio
List<HiveColumnHandle> partitionColumns = partitions.getPartitionColumns();
enforcedConstraint = partitions.getEffectivePredicate().filter((column, domain) -> partitionColumns.contains(column));
}
return new HiveTableHandle(
handle.getSchemaName(),
handle.getTableName(),
handle.getTableParameters(),
ImmutableList.copyOf(partitions.getPartitionColumns()),
handle.getDataColumns(),
partitionNames,
partitionList,
partitions.getCompactEffectivePredicate(),
enforcedConstraint,
partitions.getBucketHandle(),
partitions.getBucketFilter(),
handle.getAnalyzePartitionValues(),
Sets.union(handle.getConstraintColumns(), constraint.getPredicateColumns().orElseGet(ImmutableSet::of)),
handle.getProjectedColumns(),
handle.getTransaction(),
handle.isRecordScannedFiles(),
handle.getMaxScannedFileSize());
return HiveTableHandle.buildFrom(handle)
.withPartitionColumns(ImmutableList.copyOf(partitions.getPartitionColumns()))
.withPartitionNames(partitionNames)
.withPartitions(partitionList)
.withCompactEffectivePredicate(partitions.getCompactEffectivePredicate())
.withEnforcedConstraint(enforcedConstraint)
.withBucketHandle(partitions.getBucketHandle())
.withBucketFilter(partitions.getBucketFilter())
.withConstraintColumns(Sets.union(handle.getConstraintColumns(), constraint.getPredicateColumns().orElseGet(ImmutableSet::of)))
.build();
}

public Iterator<HivePartition> getPartitions(SemiTransactionalHiveMetastore metastore, HiveTableHandle table)
Expand Down
Loading

0 comments on commit 3350aa8

Please sign in to comment.