-
Notifications
You must be signed in to change notification settings - Fork 5.5k
Add statistics handling to the Alluxio metastore support #14018
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -15,23 +15,27 @@ | |
|
|
||
| import alluxio.client.table.TableMasterClient; | ||
| import alluxio.exception.status.AlluxioStatusException; | ||
| import alluxio.grpc.table.ColumnStatisticsInfo; | ||
| import alluxio.grpc.table.Constraint; | ||
| import alluxio.grpc.table.layout.hive.PartitionInfo; | ||
| import com.facebook.presto.hive.HiveBasicStatistics; | ||
| import com.facebook.presto.hive.HiveType; | ||
| import com.facebook.presto.hive.metastore.Column; | ||
| import com.facebook.presto.hive.metastore.Database; | ||
| import com.facebook.presto.hive.metastore.ExtendedHiveMetastore; | ||
| import com.facebook.presto.hive.metastore.HiveColumnStatistics; | ||
| import com.facebook.presto.hive.metastore.HivePrivilegeInfo; | ||
| import com.facebook.presto.hive.metastore.MetastoreUtil; | ||
| import com.facebook.presto.hive.metastore.Partition; | ||
| import com.facebook.presto.hive.metastore.PartitionStatistics; | ||
| import com.facebook.presto.hive.metastore.PartitionWithStatistics; | ||
| import com.facebook.presto.hive.metastore.PrincipalPrivileges; | ||
| import com.facebook.presto.hive.metastore.Table; | ||
| import com.facebook.presto.hive.metastore.thrift.HiveMetastore; | ||
| import com.facebook.presto.hive.metastore.thrift.ThriftMetastoreUtil; | ||
| import com.facebook.presto.spi.NotFoundException; | ||
| import com.facebook.presto.spi.PrestoException; | ||
| import com.facebook.presto.spi.SchemaTableName; | ||
| import com.facebook.presto.spi.TableNotFoundException; | ||
| import com.facebook.presto.spi.predicate.Domain; | ||
| import com.facebook.presto.spi.security.PrestoPrincipal; | ||
| import com.facebook.presto.spi.security.RoleGrant; | ||
|
|
@@ -45,15 +49,16 @@ | |
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Optional; | ||
| import java.util.OptionalLong; | ||
| import java.util.Set; | ||
| import java.util.function.Function; | ||
|
|
||
| import static com.facebook.presto.hive.MetastoreErrorCode.HIVE_METASTORE_ERROR; | ||
| import static com.facebook.presto.hive.metastore.MetastoreUtil.convertPredicateToParts; | ||
| import static com.facebook.presto.hive.metastore.MetastoreUtil.getHiveBasicStatistics; | ||
| import static com.google.common.collect.ImmutableList.toImmutableList; | ||
| import static com.google.common.collect.ImmutableMap.toImmutableMap; | ||
| import static java.util.Objects.requireNonNull; | ||
| import static java.util.function.Function.identity; | ||
|
|
||
| /** | ||
| * Implementation of the {@link HiveMetastore} interface through Alluxio. | ||
|
|
@@ -108,19 +113,26 @@ public Optional<Table> getTable(String databaseName, String tableName) | |
| @Override | ||
| public Set<ColumnStatisticType> getSupportedColumnStatistics(Type type) | ||
| { | ||
| throw new UnsupportedOperationException("getSupportedColumnStatistics is not supported in AlluxioHiveMetastore"); | ||
| return MetastoreUtil.getSupportedColumnStatistics(type); | ||
| } | ||
|
|
||
| private Map<String, HiveColumnStatistics> groupStatisticsByColumn(List<ColumnStatisticsInfo> statistics, OptionalLong rowCount) | ||
| { | ||
| return statistics.stream() | ||
| .collect(toImmutableMap(ColumnStatisticsInfo::getColName, statisticsInfo -> AlluxioProtoUtils.fromProto(statisticsInfo.getData(), rowCount))); | ||
| } | ||
|
|
||
| @Override | ||
| public PartitionStatistics getTableStatistics(String databaseName, String tableName) | ||
| { | ||
| try { | ||
| Table table = getTable(databaseName, tableName).orElseThrow(() -> new PrestoException( | ||
| HIVE_METASTORE_ERROR, | ||
| String.format("Could not retrieve table %s.%s", databaseName, tableName))); | ||
| HiveBasicStatistics basicStats = ThriftMetastoreUtil.getHiveBasicStatistics(table.getParameters()); | ||
| // TODO implement logic to populate Map<string, HiveColumnStatistics> | ||
| return new PartitionStatistics(basicStats, ImmutableMap.of()); | ||
| Table table = getTable(databaseName, tableName).orElseThrow( | ||
| () -> new PrestoException(HIVE_METASTORE_ERROR, String.format("Could not retrieve table %s.%s", databaseName, tableName))); | ||
| HiveBasicStatistics basicStatistics = getHiveBasicStatistics(table.getParameters()); | ||
| List<Column> columns = table.getPartitionColumns(); | ||
| List<String> columnNames = columns.stream().map(Column::getName).collect(toImmutableList()); | ||
| List<ColumnStatisticsInfo> columnStatistics = client.getTableColumnStatistics(table.getDatabaseName(), table.getTableName(), columnNames); | ||
| return new PartitionStatistics(basicStatistics, groupStatisticsByColumn(columnStatistics, basicStatistics.getRowCount())); | ||
| } | ||
| catch (Exception e) { | ||
| throw new PrestoException(HIVE_METASTORE_ERROR, e); | ||
|
|
@@ -130,9 +142,45 @@ public PartitionStatistics getTableStatistics(String databaseName, String tableN | |
| @Override | ||
| public Map<String, PartitionStatistics> getPartitionStatistics(String databaseName, String tableName, Set<String> partitionNames) | ||
| { | ||
| // TODO implement partition statistics | ||
| // currently returns a map of partitionName to empty statistics to satisfy presto requirements | ||
| return partitionNames.stream().collect(toImmutableMap(identity(), (p) -> PartitionStatistics.empty())); | ||
| Table table = getTable(databaseName, tableName).orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); | ||
|
|
||
| Map<String, HiveBasicStatistics> partitionBasicStatistics = getPartitionsByNames(databaseName, tableName, ImmutableList.copyOf(partitionNames)).entrySet().stream() | ||
| .filter(entry -> entry.getValue().isPresent()) | ||
| .collect(toImmutableMap( | ||
| entry -> MetastoreUtil.makePartName(table.getPartitionColumns(), entry.getValue().get().getValues()), | ||
| entry -> getHiveBasicStatistics(entry.getValue().get().getParameters()))); | ||
|
|
||
| Map<String, OptionalLong> partitionRowCounts = partitionBasicStatistics.entrySet().stream() | ||
| .collect(toImmutableMap(Map.Entry::getKey, entry -> entry.getValue().getRowCount())); | ||
|
|
||
| List<String> dataColumns = table.getDataColumns().stream() | ||
| .map(Column::getName) | ||
| .collect(toImmutableList()); | ||
| Map<String, List<ColumnStatisticsInfo>> columnStatisticss; | ||
| try { | ||
| columnStatisticss = client.getPartitionColumnStatistics( | ||
| table.getDatabaseName(), | ||
| table.getTableName(), | ||
| partitionBasicStatistics.keySet().stream().collect(toImmutableList()), | ||
| dataColumns); | ||
| } | ||
| catch (AlluxioStatusException e) { | ||
| throw new PrestoException(HIVE_METASTORE_ERROR, e); | ||
| } | ||
|
|
||
| Map<String, Map<String, HiveColumnStatistics>> partitionColumnStatistics = columnStatisticss.entrySet().stream() | ||
| .filter(entry -> !entry.getValue().isEmpty()) | ||
| .collect(toImmutableMap( | ||
| Map.Entry::getKey, | ||
| entry -> groupStatisticsByColumn(entry.getValue(), partitionRowCounts.getOrDefault(entry.getKey(), OptionalLong.empty())))); | ||
|
||
|
|
||
| ImmutableMap.Builder<String, PartitionStatistics> result = ImmutableMap.builder(); | ||
| for (String partitionName : partitionBasicStatistics.keySet()) { | ||
| HiveBasicStatistics basicStatistics = partitionBasicStatistics.get(partitionName); | ||
| Map<String, HiveColumnStatistics> columnStatistics = partitionColumnStatistics.getOrDefault(partitionName, ImmutableMap.of()); | ||
| result.put(partitionName, new PartitionStatistics(basicStatistics, columnStatistics)); | ||
| } | ||
| return result.build(); | ||
| } | ||
|
|
||
| @Override | ||
|
|
||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
break a line