You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/02/11 23:46:23 UTC

[GitHub] [iceberg] jackye1995 opened a new pull request #4100: AWS: support registerTable in DynamoDbCatalog

jackye1995 opened a new pull request #4100:
URL: https://github.com/apache/iceberg/pull/4100


   @arminnajafi @rajarshisarkar @singhpk234 @amogh-jahagirdar


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rajarshisarkar commented on a change in pull request #4100: AWS: support registerTable in DynamoDbCatalog

Posted by GitBox <gi...@apache.org>.
rajarshisarkar commented on a change in pull request #4100:
URL: https://github.com/apache/iceberg/pull/4100#discussion_r807908123



##########
File path: aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java
##########
@@ -440,6 +443,42 @@ public void close() throws IOException {
     closeableGroup.close();
   }
 
+  @Override
+  public Table registerTable(TableIdentifier identifier, String metadataFileLocation) {
+    Preconditions.checkArgument(isValidIdentifier(identifier),
+        "Cannot register: invalid table identifier " + identifier);
+    Preconditions.checkArgument(metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Cannot register: empty metadata file location as a table");
+    if (!namespaceExists(identifier.namespace())) {
+      throw new NoSuchNamespaceException("Cannot register: namespace %s not found", identifier.namespace());
+    }
+
+    Map<String, AttributeValue> values = tablePrimaryKey(identifier);
+    values.put(
+        toPropertyCol(BaseMetastoreTableOperations.TABLE_TYPE_PROP),
+        AttributeValue.builder()
+            .s(BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(Locale.ENGLISH))
+            .build());
+    values.put(
+        toPropertyCol(BaseMetastoreTableOperations.METADATA_LOCATION_PROP),
+        AttributeValue.builder()
+            .s(metadataFileLocation)
+            .build());
+    DynamoDbCatalog.setNewCatalogEntryMetadata(values);
+
+    try {
+      dynamo.putItem(PutItemRequest.builder()
+          .tableName(awsProperties.dynamoDbTableName())
+          .item(values)
+          .conditionExpression("attribute_not_exists(" + COL_VERSION + ")")
+          .build());
+    } catch (ConditionalCheckFailedException e) {
+      throw new AlreadyExistsException(e, "Cannot register: table %s already exists", identifier);
+    }

Review comment:
       Should we have a chained catch for `DynamoDbException` (Base class for all service exceptions)? `putItems` seems to throw many other exceptions apart from `ConditionalCheckFailedException`.

##########
File path: aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java
##########
@@ -440,6 +443,42 @@ public void close() throws IOException {
     closeableGroup.close();
   }
 
+  @Override
+  public Table registerTable(TableIdentifier identifier, String metadataFileLocation) {
+    Preconditions.checkArgument(isValidIdentifier(identifier),
+        "Cannot register: invalid table identifier " + identifier);

Review comment:
       `isValidIdentifier` for DynamoDB always returns true. Is this something we can implement (maybe in a separate PR)?

##########
File path: aws/src/integration/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java
##########
@@ -295,6 +302,77 @@ public void testDropNamespace() {
     Assert.assertFalse("namespace must not exist", response.hasItem());
   }
 
+  @Test
+  public void testRegisterTable() {
+    Namespace namespace = Namespace.of(genRandomName());
+    catalog.createNamespace(namespace);
+    TableIdentifier tableIdentifier = TableIdentifier.of(namespace, genRandomName());
+    catalog.createTable(tableIdentifier, SCHEMA);
+    Table table = catalog.loadTable(tableIdentifier);
+    DataFile dataFile = DataFiles.builder(PartitionSpec.unpartitioned())
+        .withPath("/path/to/data-a.parquet")
+        .withFileSizeInBytes(10)
+        .withRecordCount(1)
+        .build();
+    table.newAppend().appendFile(dataFile).commit();
+    table.refresh();
+    long v1SnapshotId = table.currentSnapshot().snapshotId();
+    String v1MetadataLocation = ((BaseTable) table).operations().current().metadataFileLocation();
+    table.newDelete().deleteFile(dataFile).commit();
+    table.refresh();

Review comment:
       I think we can avoid the refresh calls.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org