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



##########
File path: aws/src/main/java/org/apache/iceberg/aws/glue/DynamoLockManager.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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 java.util.UUID;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.aws.AwsClientFactories;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Tasks;
+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.ConditionalCheckFailedException;
+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.GetItemRequest;
+import software.amazon.awssdk.services.dynamodb.model.GetItemResponse;
+import 
software.amazon.awssdk.services.dynamodb.model.InternalServerErrorException;
+import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement;
+import software.amazon.awssdk.services.dynamodb.model.KeyType;
+import 
software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughputExceededException;
+import software.amazon.awssdk.services.dynamodb.model.PutItemRequest;
+import 
software.amazon.awssdk.services.dynamodb.model.RequestLimitExceededException;
+import 
software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType;
+import software.amazon.awssdk.services.dynamodb.model.TableStatus;
+import 
software.amazon.awssdk.services.dynamodb.model.TransactionConflictException;
+
+/**
+ * DynamoDB implementation for the lock manager.
+ */
+class DynamoLockManager extends LockManagers.LockManagerBase {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(DynamoLockManager.class);
+
+  private static final String COL_LOCK_ENTITY_ID = "entityId";
+  private static final String COL_LEASE_DURATION_MS = "leaseDurationMs";
+  private static final String COL_VERSION = "version";
+  private static final String COL_LOCK_OWNER_ID = "ownerId";
+
+  private static final String CONDITION_LOCK_ID_MATCH =
+          COL_LOCK_ENTITY_ID + "= :eid AND " +
+          COL_LOCK_OWNER_ID + " = :oid";
+  private static final String CONDITION_LOCK_ENTITY_NOT_EXIST = 
"attribute_not_exists(" + COL_LOCK_ENTITY_ID + ")";
+  private static final String CONDITION_LOCK_ENTITY_NOT_EXIST_OR_VERSION_MATCH 
=
+          "attribute_not_exists(" + COL_LOCK_ENTITY_ID + ") OR (" +
+          COL_LOCK_ENTITY_ID + "= :eid AND " +
+          COL_VERSION + "= :vid)";
+
+  private static final int LOCK_TABLE_CREATION_WAIT_ATTEMPTS_MAX = 5;
+
+  private static final List<KeySchemaElement> LOCK_TABLE_SCHEMA = 
Lists.newArrayList(
+      KeySchemaElement.builder()
+          .attributeName(COL_LOCK_ENTITY_ID)
+          .keyType(KeyType.HASH)
+          .build()
+  );
+
+  private static final List<AttributeDefinition> LOCK_TABLE_COL_DEFINITIONS = 
Lists.newArrayList(
+      AttributeDefinition.builder()
+          .attributeName(COL_LOCK_ENTITY_ID)
+          .attributeType(ScalarAttributeType.S)
+          .build()
+  );
+
+  private final Map<String, ScheduledFuture<?>> heartbeats = Maps.newHashMap();
+
+  private DynamoDbClient dynamo;
+  private String lockTableName;
+
+  /**
+   * constructor for dynamic initialization, {@link #initialize(Map)} must be 
called later.
+   */
+  DynamoLockManager() {
+  }
+
+  /**
+   * constructor used for testing purpose
+   * @param dynamo dynamo client
+   * @param lockTableName lock table name
+   */
+  DynamoLockManager(DynamoDbClient dynamo, String lockTableName) {
+    super.initialize(Maps.newHashMap());
+    this.dynamo = dynamo;
+    this.lockTableName = lockTableName;
+    ensureLockTableExistsOrCreate();
+  }
+
+  private void ensureLockTableExistsOrCreate() {
+
+    if (tableExists(lockTableName)) {
+      return;
+    }
+
+    LOG.info("Dynamo lock table {} not found, trying to create", 
lockTableName);
+    dynamo.createTable(CreateTableRequest.builder()
+        .tableName(lockTableName)
+        .keySchema(lockTableSchema())
+        .attributeDefinitions(lockTableColDefinitions())
+        .billingMode(BillingMode.PAY_PER_REQUEST)
+        .build());
+
+    Tasks.foreach(lockTableName)
+        .retry(LOCK_TABLE_CREATION_WAIT_ATTEMPTS_MAX)
+        .throwFailureWhenFinished()
+        .onlyRetryOn(IllegalStateException.class)
+        .run(this::checkTableActive);
+  }
+
+  @VisibleForTesting
+  boolean tableExists(String tableName) {
+    try {
+      dynamo.describeTable(DescribeTableRequest.builder()
+          .tableName(tableName)
+          .build());
+      return true;
+    } catch (ResourceNotFoundException e) {
+      return false;
+    }
+  }
+
+  private void checkTableActive(String tableName) {
+    try {
+      DescribeTableResponse response = 
dynamo.describeTable(DescribeTableRequest.builder()
+          .tableName(tableName)
+          .build());
+      TableStatus currentStatus = response.table().tableStatus();
+      if (!currentStatus.equals(TableStatus.ACTIVE)) {
+        throw new IllegalStateException(String.format("Dynamo table %s is not 
active, current status: %s",
+            tableName, currentStatus));
+      }
+    } catch (ResourceNotFoundException e) {
+      throw new IllegalStateException(String.format("Cannot find Dynamo table 
%s", tableName));
+    }
+  }
+
+  @Override
+  public void initialize(Map<String, String> properties) {
+    super.initialize(properties);
+    this.dynamo = AwsClientFactories.from(properties).dynamo();
+    this.lockTableName = properties.get(CatalogProperties.LOCK_TABLE);
+    Preconditions.checkNotNull(lockTableName, "DynamoDB lock table name must 
not be null");
+    ensureLockTableExistsOrCreate();
+  }
+
+  @Override
+  public void acquire(LockId lockId) {
+    try {
+      Tasks.foreach(lockId)
+          .throwFailureWhenFinished()
+          .retry(Integer.MAX_VALUE - 1)
+          .exponentialBackoff(acquireIntervalMs(), acquireIntervalMs(), 
acquireTimeoutMs(), 1)
+          .onlyRetryOn(
+              ConditionalCheckFailedException.class,
+              ProvisionedThroughputExceededException.class,
+              TransactionConflictException.class,
+              RequestLimitExceededException.class,
+              InternalServerErrorException.class)
+          .run(this::acquireOnce);
+    } catch (RuntimeException e) {
+      throw new LockNotAcquiredException("Fail to acquire lock " + lockId, e);
+    }
+  }
+
+  @VisibleForTesting
+  void acquireOnce(LockId lockId) {
+    GetItemResponse response = dynamo.getItem(GetItemRequest.builder()
+        .tableName(lockTableName)
+        .key(toKey(lockId))
+        .build());
+
+    if (!response.hasItem()) {
+      dynamo.putItem(PutItemRequest.builder()
+          .tableName(lockTableName)
+          .item(toNewItem(lockId))
+          .conditionExpression(CONDITION_LOCK_ENTITY_NOT_EXIST)
+          .build());
+    } else {
+      Map<String, AttributeValue> currentItem = response.item();
+
+      try {
+        
Thread.sleep(Long.parseLong(currentItem.get(COL_LEASE_DURATION_MS).n()));
+      } catch (InterruptedException e) {
+        throw new LockNotAcquiredException(
+                String.format("Fail to acquire lock %s, interrupted during 
sleep", lockId), e);
+      }
+
+      Map<String, AttributeValue> expressionValues = Maps.newHashMap();
+      expressionValues.put(":eid", 
AttributeValue.builder().s(lockId.entityId()).build());
+      expressionValues.put(":vid", 
AttributeValue.builder().s(currentItem.get(COL_VERSION).s()).build());

Review comment:
       Can you also store the interval in the row? That way, the process that 
holds the lock shares its expectation with other processes. If you don't do 
that, then another process could use a wait interval shorter than the lock 
holder's heartbeat interval and incorrectly unlock. So the expected wait 
interval should come from the process that currently holds the lock, not the 
process attempting to get it.




----------------------------------------------------------------
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:
us...@infra.apache.org



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

Reply via email to