aokolnychyi commented on issue #366: Concurrent appends to the same table using 
HiveCatalog lead to deadlock in the Metastore
URL: 
https://github.com/apache/incubator-iceberg/issues/366#issuecomment-522038874
 
 
   HMS has an internal mechanism for recovering in such cases, which might be 
slow in highly concurrent environments. One point that concerns me a bit is 
that we constantly send check lock requests without any time between requests, 
which puts quite some pressure on HMS (see `HiveTablesOperations$acquireLock`).
   
   The following succeeds in 2-20 seconds after adding a sleep between check 
lock requests (the test is identical to what @rdsr had, except table 
properties):
   
   ```
       Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
   
       icebergTable.updateProperties()
           .set(TableProperties.COMMIT_NUM_RETRIES, "10")
           .set(TableProperties.COMMIT_MIN_RETRY_WAIT_MS, "50")
           .set(TableProperties.COMMIT_MAX_RETRY_WAIT_MS, "10")
           .commit();
   
       String fileName = UUID.randomUUID().toString();
       DataFile file = DataFiles.builder(icebergTable.spec())
           .withPath(FileFormat.PARQUET.addExtension(fileName))
           .withRecordCount(2)
           .withFileSizeInBytes(0)
           .build();
   
       // If we reduce the number of threads, the problem is less evident.
       ExecutorService executorService = 
MoreExecutors.getExitingExecutorService(
           (ThreadPoolExecutor) Executors.newFixedThreadPool(10));
   
       Consumer<Integer> t = (idx) -> {
         long startTime = System.currentTimeMillis();
         System.out.println("Starting append " + idx);
         icebergTable.newAppend().appendFile(file).commit();
         System.out.println("Time taken for append " + idx + "in thread " + 
Thread.currentThread() + " = "
             + (System.currentTimeMillis() - startTime + "ms"));
       };
   
       for (int i = 0; i < 10; i++) {
         int finalI = i;
         executorService.submit(() -> t.accept(finalI));
       }
   
       executorService.shutdown();
       executorService.awaitTermination(1, TimeUnit.MINUTES);
   ```

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to