nsivabalan commented on code in PR #8658:
URL: https://github.com/apache/hudi/pull/8658#discussion_r1186893387


##########
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java:
##########
@@ -52,6 +54,107 @@ public TestInProcessLockProvider() {
     lockConfiguration2 = new LockConfiguration(properties);
   }
 
+  @Test
+  public void testLockIdentity() throws InterruptedException {
+    // The lifecycle of an InProcessLockProvider should not affect the 
singleton lock
+    // for a single table, i.e., all three writers should hold the same 
underlying lock instance
+    // on the same table.
+    // Writer 1:   lock |----------------| unlock and close
+    // Writer 2:   try lock   |      ...    lock |------| unlock and close
+    // Writer 3:                          try lock  | ...  lock |------| 
unlock and close
+    List<InProcessLockProvider> lockProviderList = new ArrayList<>();
+    InProcessLockProvider lockProvider1 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+    lockProviderList.add(lockProvider1);
+    AtomicBoolean writer2Locked = new AtomicBoolean(false);
+    AtomicBoolean writer1Completed = new AtomicBoolean(false);
+    AtomicBoolean writer2Completed = new AtomicBoolean(false);
+    AtomicBoolean writer3Completed = new AtomicBoolean(false);
+
+    // Writer 1
+    assertDoesNotThrow(() -> {
+      LOG.info("Writer 1 tries to acquire the lock.");
+      lockProvider1.lock();
+      LOG.info("Writer 1 acquires the lock.");
+    });
+    // Writer 2 thread in parallel, should block
+    // and later acquire the lock once it is released
+    Thread writer2 = new Thread(() -> {
+      InProcessLockProvider lockProvider2 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+      lockProviderList.add(lockProvider2);
+      assertDoesNotThrow(() -> {
+        LOG.info("Writer 2 tries to acquire the lock.");
+        lockProvider2.lock();
+        LOG.info("Writer 2 acquires the lock.");

Review Comment:
   can we assert that none of writer1 and writer3 is acquiring the lock here?



##########
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java:
##########
@@ -52,6 +54,107 @@ public TestInProcessLockProvider() {
     lockConfiguration2 = new LockConfiguration(properties);
   }
 
+  @Test
+  public void testLockIdentity() throws InterruptedException {
+    // The lifecycle of an InProcessLockProvider should not affect the 
singleton lock
+    // for a single table, i.e., all three writers should hold the same 
underlying lock instance
+    // on the same table.
+    // Writer 1:   lock |----------------| unlock and close
+    // Writer 2:   try lock   |      ...    lock |------| unlock and close
+    // Writer 3:                          try lock  | ...  lock |------| 
unlock and close
+    List<InProcessLockProvider> lockProviderList = new ArrayList<>();
+    InProcessLockProvider lockProvider1 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+    lockProviderList.add(lockProvider1);
+    AtomicBoolean writer2Locked = new AtomicBoolean(false);
+    AtomicBoolean writer1Completed = new AtomicBoolean(false);
+    AtomicBoolean writer2Completed = new AtomicBoolean(false);
+    AtomicBoolean writer3Completed = new AtomicBoolean(false);
+
+    // Writer 1
+    assertDoesNotThrow(() -> {
+      LOG.info("Writer 1 tries to acquire the lock.");
+      lockProvider1.lock();
+      LOG.info("Writer 1 acquires the lock.");
+    });
+    // Writer 2 thread in parallel, should block
+    // and later acquire the lock once it is released
+    Thread writer2 = new Thread(() -> {
+      InProcessLockProvider lockProvider2 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+      lockProviderList.add(lockProvider2);
+      assertDoesNotThrow(() -> {
+        LOG.info("Writer 2 tries to acquire the lock.");
+        lockProvider2.lock();
+        LOG.info("Writer 2 acquires the lock.");
+      });
+      writer2Locked.set(true);
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      assertDoesNotThrow(() -> {
+        lockProvider2.unlock();
+        LOG.info("Writer 2 releases the lock.");
+      });
+      lockProvider2.close();
+      LOG.info("Writer 2 closes the lock provider.");
+      writer2Completed.set(true);
+    });
+
+    Thread writer3 = new Thread(() -> {
+      while (!writer2Locked.get() || !writer1Completed.get()) {
+        try {
+          Thread.sleep(10);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+      }
+
+      InProcessLockProvider lockProvider3 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+      lockProviderList.add(lockProvider3);
+      assertDoesNotThrow(() -> {
+        LOG.info("Writer 3 tries to acquire the lock.");
+        lockProvider3.lock();
+        LOG.info("Writer 3 acquires the lock.");
+      });
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      assertDoesNotThrow(() -> {
+        lockProvider3.unlock();
+        LOG.info("Writer 3 releases the lock.");
+      });
+      lockProvider3.close();
+      LOG.info("Writer 3 closes the lock provider.");
+      writer3Completed.set(true);
+    });
+
+    writer2.start();
+    writer3.start();
+
+    Thread.sleep(1000);

Review Comment:
   why this arbitrary sleep. Can we make this deterministic?
   



##########
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java:
##########
@@ -52,6 +54,107 @@ public TestInProcessLockProvider() {
     lockConfiguration2 = new LockConfiguration(properties);
   }
 
+  @Test
+  public void testLockIdentity() throws InterruptedException {
+    // The lifecycle of an InProcessLockProvider should not affect the 
singleton lock
+    // for a single table, i.e., all three writers should hold the same 
underlying lock instance
+    // on the same table.
+    // Writer 1:   lock |----------------| unlock and close
+    // Writer 2:   try lock   |      ...    lock |------| unlock and close
+    // Writer 3:                          try lock  | ...  lock |------| 
unlock and close
+    List<InProcessLockProvider> lockProviderList = new ArrayList<>();
+    InProcessLockProvider lockProvider1 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+    lockProviderList.add(lockProvider1);
+    AtomicBoolean writer2Locked = new AtomicBoolean(false);
+    AtomicBoolean writer1Completed = new AtomicBoolean(false);
+    AtomicBoolean writer2Completed = new AtomicBoolean(false);
+    AtomicBoolean writer3Completed = new AtomicBoolean(false);
+
+    // Writer 1
+    assertDoesNotThrow(() -> {
+      LOG.info("Writer 1 tries to acquire the lock.");
+      lockProvider1.lock();
+      LOG.info("Writer 1 acquires the lock.");
+    });
+    // Writer 2 thread in parallel, should block
+    // and later acquire the lock once it is released
+    Thread writer2 = new Thread(() -> {
+      InProcessLockProvider lockProvider2 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+      lockProviderList.add(lockProvider2);
+      assertDoesNotThrow(() -> {
+        LOG.info("Writer 2 tries to acquire the lock.");
+        lockProvider2.lock();
+        LOG.info("Writer 2 acquires the lock.");
+      });
+      writer2Locked.set(true);
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      assertDoesNotThrow(() -> {
+        lockProvider2.unlock();
+        LOG.info("Writer 2 releases the lock.");
+      });
+      lockProvider2.close();
+      LOG.info("Writer 2 closes the lock provider.");
+      writer2Completed.set(true);
+    });
+
+    Thread writer3 = new Thread(() -> {
+      while (!writer2Locked.get() || !writer1Completed.get()) {
+        try {
+          Thread.sleep(10);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+      }
+
+      InProcessLockProvider lockProvider3 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+      lockProviderList.add(lockProvider3);
+      assertDoesNotThrow(() -> {
+        LOG.info("Writer 3 tries to acquire the lock.");
+        lockProvider3.lock();
+        LOG.info("Writer 3 acquires the lock.");

Review Comment:
   similarly, can we assert that none of writer1 and wirter2 is acquiring the 
lock at this line of execution? 



##########
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java:
##########
@@ -52,6 +54,107 @@ public TestInProcessLockProvider() {
     lockConfiguration2 = new LockConfiguration(properties);
   }
 
+  @Test
+  public void testLockIdentity() throws InterruptedException {
+    // The lifecycle of an InProcessLockProvider should not affect the 
singleton lock
+    // for a single table, i.e., all three writers should hold the same 
underlying lock instance
+    // on the same table.
+    // Writer 1:   lock |----------------| unlock and close
+    // Writer 2:   try lock   |      ...    lock |------| unlock and close
+    // Writer 3:                          try lock  | ...  lock |------| 
unlock and close
+    List<InProcessLockProvider> lockProviderList = new ArrayList<>();
+    InProcessLockProvider lockProvider1 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+    lockProviderList.add(lockProvider1);
+    AtomicBoolean writer2Locked = new AtomicBoolean(false);
+    AtomicBoolean writer1Completed = new AtomicBoolean(false);
+    AtomicBoolean writer2Completed = new AtomicBoolean(false);
+    AtomicBoolean writer3Completed = new AtomicBoolean(false);
+
+    // Writer 1
+    assertDoesNotThrow(() -> {
+      LOG.info("Writer 1 tries to acquire the lock.");
+      lockProvider1.lock();
+      LOG.info("Writer 1 acquires the lock.");
+    });
+    // Writer 2 thread in parallel, should block
+    // and later acquire the lock once it is released
+    Thread writer2 = new Thread(() -> {
+      InProcessLockProvider lockProvider2 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+      lockProviderList.add(lockProvider2);
+      assertDoesNotThrow(() -> {
+        LOG.info("Writer 2 tries to acquire the lock.");
+        lockProvider2.lock();
+        LOG.info("Writer 2 acquires the lock.");
+      });
+      writer2Locked.set(true);
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      assertDoesNotThrow(() -> {
+        lockProvider2.unlock();
+        LOG.info("Writer 2 releases the lock.");
+      });
+      lockProvider2.close();
+      LOG.info("Writer 2 closes the lock provider.");
+      writer2Completed.set(true);
+    });
+
+    Thread writer3 = new Thread(() -> {
+      while (!writer2Locked.get() || !writer1Completed.get()) {
+        try {
+          Thread.sleep(10);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+      }
+
+      InProcessLockProvider lockProvider3 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+      lockProviderList.add(lockProvider3);
+      assertDoesNotThrow(() -> {
+        LOG.info("Writer 3 tries to acquire the lock.");
+        lockProvider3.lock();
+        LOG.info("Writer 3 acquires the lock.");
+      });
+      try {
+        Thread.sleep(1000);

Review Comment:
   can we reduce this to 500ms. Trying to reduce the total test run time.



##########
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java:
##########
@@ -52,6 +54,107 @@ public TestInProcessLockProvider() {
     lockConfiguration2 = new LockConfiguration(properties);
   }
 
+  @Test
+  public void testLockIdentity() throws InterruptedException {
+    // The lifecycle of an InProcessLockProvider should not affect the 
singleton lock
+    // for a single table, i.e., all three writers should hold the same 
underlying lock instance
+    // on the same table.
+    // Writer 1:   lock |----------------| unlock and close
+    // Writer 2:   try lock   |      ...    lock |------| unlock and close
+    // Writer 3:                          try lock  | ...  lock |------| 
unlock and close
+    List<InProcessLockProvider> lockProviderList = new ArrayList<>();
+    InProcessLockProvider lockProvider1 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+    lockProviderList.add(lockProvider1);
+    AtomicBoolean writer2Locked = new AtomicBoolean(false);
+    AtomicBoolean writer1Completed = new AtomicBoolean(false);
+    AtomicBoolean writer2Completed = new AtomicBoolean(false);
+    AtomicBoolean writer3Completed = new AtomicBoolean(false);
+
+    // Writer 1
+    assertDoesNotThrow(() -> {
+      LOG.info("Writer 1 tries to acquire the lock.");
+      lockProvider1.lock();
+      LOG.info("Writer 1 acquires the lock.");
+    });
+    // Writer 2 thread in parallel, should block
+    // and later acquire the lock once it is released
+    Thread writer2 = new Thread(() -> {
+      InProcessLockProvider lockProvider2 = new 
InProcessLockProvider(lockConfiguration1, hadoopConfiguration);
+      lockProviderList.add(lockProvider2);
+      assertDoesNotThrow(() -> {
+        LOG.info("Writer 2 tries to acquire the lock.");
+        lockProvider2.lock();
+        LOG.info("Writer 2 acquires the lock.");
+      });
+      writer2Locked.set(true);
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      assertDoesNotThrow(() -> {
+        lockProvider2.unlock();
+        LOG.info("Writer 2 releases the lock.");
+      });
+      lockProvider2.close();
+      LOG.info("Writer 2 closes the lock provider.");
+      writer2Completed.set(true);
+    });
+
+    Thread writer3 = new Thread(() -> {
+      while (!writer2Locked.get() || !writer1Completed.get()) {

Review Comment:
   why writer3 awaits until writer2 goes into lock. Shouldn't we try to get 
both writer2 and writer3 to try acquiring lock or waiting to acquire lock 
simultaneously ? 



-- 
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: commits-unsubscr...@hudi.apache.org

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

Reply via email to