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 2020/12/01 19:42:38 UTC

[GitHub] [iceberg] yyanyy commented on a change in pull request #1823: AWS: Glue catalog locking through DynamoDB

yyanyy commented on a change in pull request #1823:
URL: https://github.com/apache/iceberg/pull/1823#discussion_r533674766



##########
File path: aws/src/main/java/org/apache/iceberg/aws/glue/DynamoLockManager.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.aws.glue;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.aws.AwsClientUtil;
+import org.apache.iceberg.aws.AwsProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
+import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.BillingMode;
+import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest;
+import software.amazon.awssdk.services.dynamodb.model.DeleteItemRequest;
+import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest;
+import software.amazon.awssdk.services.dynamodb.model.DescribeTableResponse;
+import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement;
+import software.amazon.awssdk.services.dynamodb.model.KeyType;
+import software.amazon.awssdk.services.dynamodb.model.PutItemRequest;
+import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType;
+import software.amazon.awssdk.services.dynamodb.model.TableStatus;
+
+class DynamoLockManager implements LockManager {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DynamoLockManager.class);
+
+  private static final String LOCK_TABLE_COL_TABLE_ID = "tableId";
+  private static final String LOCK_TABLE_COL_EXPIRE_TS_MILLIS = "expireTimestampMillis";
+
+  private static final List<KeySchemaElement> LOCK_TABLE_SCHEMA = Lists.newArrayList(
+      KeySchemaElement.builder()
+          .attributeName(LOCK_TABLE_COL_TABLE_ID)
+          .keyType(KeyType.HASH)
+          .build()
+  );
+
+  private static final List<AttributeDefinition> LOCK_TABLE_COL_DEFINITIONS = Lists.newArrayList(
+      AttributeDefinition.builder()
+          .attributeName(LOCK_TABLE_COL_TABLE_ID)
+          .attributeType(ScalarAttributeType.S)
+          .build()
+  );
+
+  private final DynamoDbClient dynamo;
+  private final AwsProperties awsProperties;
+
+  DynamoLockManager(AwsProperties awsProperties) {
+    this(AwsClientUtil.defaultDynamoClient(), awsProperties);
+  }
+
+  DynamoLockManager(DynamoDbClient dynamo, AwsProperties awsProperties) {
+    this.dynamo = dynamo;
+    this.awsProperties = awsProperties;
+    ensureLockTableExists();
+  }
+
+  private void ensureLockTableExists() {
+    ensureTableExists(awsProperties.glueCatalogLockTable(), LOCK_TABLE_SCHEMA, LOCK_TABLE_COL_DEFINITIONS);
+  }
+
+  private void ensureTableExists(String tableName, List<KeySchemaElement> schema,
+                                 List<AttributeDefinition> definitions) {
+    try {
+      dynamo.describeTable(DescribeTableRequest.builder()
+          .tableName(tableName)
+          .build());
+    } catch (ResourceNotFoundException e) {
+      LOG.info("Glue lock DynamoDB table <{}> not found, trying to create", tableName);
+      dynamo.createTable(CreateTableRequest.builder()
+          .tableName(tableName)
+          .keySchema(schema)
+          .attributeDefinitions(definitions)
+          .billingMode(BillingMode.PAY_PER_REQUEST)
+          .build());
+
+      boolean isTableActive = false;
+      while (!isTableActive) {
+        LOG.info("waiting for DynamoDB table <{}> to be active", tableName);
+        try {
+          Thread.sleep(5000);
+        } catch (InterruptedException ie) {
+          LOG.warn("Glue lock DynamoDB table creation sleep interrupted", e);
+        }
+        DescribeTableResponse describeTableResponse = dynamo.describeTable(DescribeTableRequest.builder()
+            .tableName(tableName)
+            .build());
+        isTableActive = describeTableResponse.table().tableStatus().equals(TableStatus.ACTIVE);
+      }
+    }
+  }
+
+
+  @Override
+  public boolean tryLock(String database, String table, long expireMillis) {
+    String tableId = tableId(database, table);
+
+    Map<String, AttributeValue> key = Maps.newHashMap();
+    key.put(LOCK_TABLE_COL_TABLE_ID, AttributeValue.builder().s(tableId).build());
+    key.put(LOCK_TABLE_COL_EXPIRE_TS_MILLIS, AttributeValue.builder().n(
+        Long.toString(System.currentTimeMillis() + expireMillis)

Review comment:
       I think there's no guarantee on how far away the clock can skew, so 20min would reduce the chance but wouldn't fully avoid the problem. Regarding its explanation, I think in addition to expiration time it also has the concept of heartbeat which has a shorter duration than the expiration time, and the lock holding thread will need to heartbeat and update the GUID once a while to extend the lease, so that when another thread observes a GUID, wait for the expiration time, and come back and still see the same GUID, then that definitely means the old thread died. In this case we probably don't want to set expiration time to be as long as 20min since that would mean if a thread died even without concurrent thread waiting, the next coming thread will have to wait for 20min before considering the lock as expired. 




----------------------------------------------------------------
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.

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