Skip to content

Commit

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

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

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();
return new HiveTableHandle(
tableName.getSchemaName(),
tableName.getTableName(),
table.getParameters(),
getPartitionKeyColumnHandles(table, typeManager),
getRegularColumnHandles(table, typeManager, getTimestampPrecision(session)),
getHiveBucketHandle(session, table, typeManager));
}

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

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

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

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

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

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

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

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

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

List<Assignment> outputAssignments = ImmutableList.copyOf(newAssignments.values());
return Optional.of(new ProjectionApplicationResult<>(
HiveTableHandle.buildFrom(hiveTableHandle)
.withProjectedColumns(projectedColumnsBuilder.build())
.build(),
hiveTableHandle.withProjectedColumns(projectedColumnsBuilder.build()),
newProjections,
outputAssignments,
false));
Expand Down Expand Up @@ -3229,16 +3213,29 @@ 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 HiveTableHandle.buildFrom(hiveTable)
.withBucketHandle(Optional.of(new HiveBucketHandle(
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(
bucketHandle.getColumns(),
bucketHandle.getBucketingVersion(),
bucketHandle.getTableBucketCount(),
hivePartitioningHandle.getBucketCount(),
bucketHandle.getSortedBy())))
.withConstraintColumns(ImmutableSet.of())
.withProjectedColumns(ImmutableSet.of()) // Projected columns is used only during optimization phase of planning
.build();
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());
}

@VisibleForTesting
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,16 +168,24 @@ public HiveTableHandle applyPartitionResult(HiveTableHandle handle, HivePartitio
List<HiveColumnHandle> partitionColumns = partitions.getPartitionColumns();
enforcedConstraint = partitions.getEffectivePredicate().filter((column, domain) -> partitionColumns.contains(column));
}
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();
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());
}

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

0 comments on commit 4a43f9b

Please sign in to comment.