You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by xu...@apache.org on 2022/04/14 04:03:10 UTC
[hudi] branch master updated: [HUDI-3876] Fixing fetching partitions in GlueSyncClient (#5318)
This is an automated email from the ASF dual-hosted git repository.
xushiyan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git
The following commit(s) were added to refs/heads/master by this push:
new a081c2b9b5 [HUDI-3876] Fixing fetching partitions in GlueSyncClient (#5318)
a081c2b9b5 is described below
commit a081c2b9b54dcf5720230a2c7e0229c765208e24
Author: Sivabalan Narayanan <n....@gmail.com>
AuthorDate: Thu Apr 14 00:03:05 2022 -0400
[HUDI-3876] Fixing fetching partitions in GlueSyncClient (#5318)
---
.../hudi/aws/sync/AWSGlueCatalogSyncClient.java | 20 +++++++++++++-------
1 file changed, 13 insertions(+), 7 deletions(-)
diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java
index 97e47deed8..81c05ed132 100644
--- a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java
+++ b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java
@@ -94,13 +94,19 @@ public class AWSGlueCatalogSyncClient extends AbstractHiveSyncHoodieClient {
@Override
public List<Partition> getAllPartitions(String tableName) {
try {
- GetPartitionsRequest request = new GetPartitionsRequest();
- request.withDatabaseName(databaseName).withTableName(tableName);
- GetPartitionsResult result = awsGlue.getPartitions(request);
- return result.getPartitions()
- .stream()
- .map(p -> new Partition(p.getValues(), p.getStorageDescriptor().getLocation()))
- .collect(Collectors.toList());
+ List<Partition> partitions = new ArrayList<>();
+ String nextToken = null;
+ do {
+ GetPartitionsResult result = awsGlue.getPartitions(new GetPartitionsRequest()
+ .withDatabaseName(databaseName)
+ .withTableName(tableName)
+ .withNextToken(nextToken));
+ partitions.addAll(result.getPartitions().stream()
+ .map(p -> new Partition(p.getValues(), p.getStorageDescriptor().getLocation()))
+ .collect(Collectors.toList()));
+ nextToken = result.getNextToken();
+ } while (nextToken != null);
+ return partitions;
} catch (Exception e) {
throw new HoodieGlueSyncException("Failed to get all partitions for table " + tableId(databaseName, tableName), e);
}