You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "GabeChurch (via GitHub)" <gi...@apache.org> on 2023/01/26 19:57:35 UTC

[GitHub] [iceberg] GabeChurch commented on issue #6667: Spark Hive Iceberg Table Locks -- Settings Unclear in Docs + Overrides Not Working

GabeChurch commented on issue #6667:
URL: https://github.com/apache/iceberg/issues/6667#issuecomment-1405560496

   Thanks for the quick response @pvary just saw that ticket last night as well! 
   
   I actually created a benchmark in airflow, that runs 30 concurrent Spark jobs with 50 sequential writes (of 400GB) per job to fine tune settings and benchmark iceberg behavior on "high frequency writes with Spark". With a single job I am able to achieve 8mins for 50 (separate) sequential writes to the table. 
   
   Based on the initial logs it was somewhat unclear to me that I could control commit retries to and hive locks were not actually the point of failure on commit.
   
   ```[2023-01-24 23:20:39,050] {spark_submit.py:536} INFO - 23/01/24 23:20:39 WARN Tasks: Retrying task after failure: Waiting for lock on table performance_tests.parallel_test2
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - org.apache.iceberg.hive.HiveTableOperations$WaitingForLockException: Waiting for lock on table performance_tests.parallel_test2
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.hive.HiveTableOperations.lambda$acquireLock$10(HiveTableOperations.java:653)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.hive.HiveTableOperations.acquireLock(HiveTableOperations.java:645)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.hive.HiveTableOperations.doCommit(HiveTableOperations.java:266)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.BaseMetastoreTableOperations.commit(BaseMetastoreTableOperations.java:135)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.SnapshotProducer.lambda$commit$2(SnapshotProducer.java:391)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.SnapshotProducer.commit(SnapshotProducer.java:365)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.spark.source.SparkWrite.commitOperation(SparkWrite.java:215)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.spark.source.SparkWrite.access$1300(SparkWrite.java:97)
   [2023-01-24 23:20:39,051] {spark_submit.py:536} INFO - at org.apache.iceberg.spark.source.SparkWrite$BatchAppend.commit(SparkWrite.java:295)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2(WriteToDataSourceV2Exec.scala:392)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2$(WriteToDataSourceV2Exec.scala:353)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.AppendDataExec.writeWithV2(WriteToDataSourceV2Exec.scala:244)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2ExistingTableWriteExec.run(WriteToDataSourceV2Exec.scala:332)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2ExistingTableWriteExec.run$(WriteToDataSourceV2Exec.scala:331)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.AppendDataExec.run(WriteToDataSourceV2Exec.scala:244)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:98)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:109)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:169)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:95)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:94)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:584)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:176)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:584)
   [2023-01-24 23:20:39,052] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:560)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:94)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:81)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:79)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:116)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at org.apache.spark.sql.DataFrameWriterV2.runCommand(DataFrameWriterV2.scala:195)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at org.apache.spark.sql.DataFrameWriterV2.append(DataFrameWriterV2.scala:149)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at py4j.Gateway.invoke(Gateway.java:282)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at py4j.commands.CallCommand.execute(CallCommand.java:79)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:182)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at py4j.ClientServerConnection.run(ClientServerConnection.java:106)
   [2023-01-24 23:20:39,053] {spark_submit.py:536} INFO - at java.base/java.lang.Thread.run(Thread.java:829)
   [2023-01-24 23:20:44,859] {spark_submit.py:536} INFO - 23/01/24 23:20:44 ERROR AppendDataExec: Data source write support IcebergBatchWrite(table=iceberg.performance_tests.parallel_test2, format=ORC) is aborting.
   [2023-01-24 23:20:45,245] {spark_submit.py:536} INFO - 23/01/24 23:20:45 ERROR AppendDataExec: Data source write support IcebergBatchWrite(table=iceberg.performance_tests.parallel_test2, format=ORC) aborted.
   [2023-01-24 23:20:45,340] {spark_submit.py:536} INFO - Traceback (most recent call last):
   [2023-01-24 23:20:45,341] {spark_submit.py:536} INFO - File "/tmp/spark-bd465075-431a-4b39-b40e-6aafefde4fd3/write_1hz_sample.py", line 121, in <module>
   [2023-01-24 23:20:45,341] {spark_submit.py:536} INFO - raise e
   [2023-01-24 23:20:45,341] {spark_submit.py:536} INFO - File "/tmp/spark-bd465075-431a-4b39-b40e-6aafefde4fd3/write_1hz_sample.py", line 96, in <module>
   [2023-01-24 23:20:45,341] {spark_submit.py:536} INFO - sample_spark_df.sortWithinPartitions(col('timestamp'), col('group_id')).writeTo(hive_table) \
   [2023-01-24 23:20:45,341] {spark_submit.py:536} INFO - File "/opt/spark/python/pyspark/sql/readwriter.py", line 1460, in append
   [2023-01-24 23:20:45,341] {spark_submit.py:536} INFO - self._jwriter.append()
   [2023-01-24 23:20:45,341] {spark_submit.py:536} INFO - File "/opt/spark/python/lib/py4j-0.10.9.5-src.zip/py4j/java_gateway.py", line 1321, in __call__
   [2023-01-24 23:20:45,341] {spark_submit.py:536} INFO - File "/opt/spark/python/pyspark/sql/utils.py", line 190, in deco
   [2023-01-24 23:20:45,341] {spark_submit.py:536} INFO - return f(*a, **kw)
   [2023-01-24 23:20:45,341] {spark_submit.py:536} INFO - File "/opt/spark/python/lib/py4j-0.10.9.5-src.zip/py4j/protocol.py", line 326, in get_return_value
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - py4j.protocol.Py4JJavaError: An error occurred while calling o100.append.
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - : org.apache.spark.SparkException: Writing job aborted
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.errors.QueryExecutionErrors$.writingJobAbortedError(QueryExecutionErrors.scala:767)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2(WriteToDataSourceV2Exec.scala:409)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2$(WriteToDataSourceV2Exec.scala:353)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.AppendDataExec.writeWithV2(WriteToDataSourceV2Exec.scala:244)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2ExistingTableWriteExec.run(WriteToDataSourceV2Exec.scala:332)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2ExistingTableWriteExec.run$(WriteToDataSourceV2Exec.scala:331)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.AppendDataExec.run(WriteToDataSourceV2Exec.scala:244)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:98)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:109)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:169)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:95)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:94)
   [2023-01-24 23:20:45,342] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:584)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:176)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:584)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:560)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:94)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:81)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:79)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:116)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.DataFrameWriterV2.runCommand(DataFrameWriterV2.scala:195)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at org.apache.spark.sql.DataFrameWriterV2.append(DataFrameWriterV2.scala:149)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
   [2023-01-24 23:20:45,343] {spark_submit.py:536} INFO - at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at py4j.Gateway.invoke(Gateway.java:282)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at py4j.commands.CallCommand.execute(CallCommand.java:79)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:182)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at py4j.ClientServerConnection.run(ClientServerConnection.java:106)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at java.base/java.lang.Thread.run(Thread.java:829)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - Caused by: org.apache.iceberg.exceptions.CommitFailedException: Base metadata location 's3a://iceberg_tests/orc_parallel_test2/metadata/00998-0dbd9354-f168-414a-9ac3-3281da97fdf7.metadata.json' is not same as the current table metadata location 's3a://iceberg_tests/orc_parallel_test2/metadata/00999-601e7061-e876-45ee-82ab-176bc1b9ade5.metadata.json' for performance_tests.parallel_test2
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.iceberg.hive.HiveTableOperations.doCommit(HiveTableOperations.java:294)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.iceberg.BaseMetastoreTableOperations.commit(BaseMetastoreTableOperations.java:135)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.iceberg.SnapshotProducer.lambda$commit$2(SnapshotProducer.java:391)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.iceberg.SnapshotProducer.commit(SnapshotProducer.java:365)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.iceberg.spark.source.SparkWrite.commitOperation(SparkWrite.java:215)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.iceberg.spark.source.SparkWrite.access$1300(SparkWrite.java:97)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.iceberg.spark.source.SparkWrite$BatchAppend.commit(SparkWrite.java:295)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - at org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2(WriteToDataSourceV2Exec.scala:392)
   [2023-01-24 23:20:45,344] {spark_submit.py:536} INFO - ... 43 more
   
   ```
   
   
   I realized shortly after posting my original question that Hive locks weren't the only issue, but it appears what I saw was actually a combination of metadata changing between writes, locks being released, and commit retries failing after hitting the default failure limit of 4. I will probably make a simple M/R to extend org.apache.iceberg.exceptions.CommitFailedException that includes the number of retries so it is obvious when retry limit is hit -- making it more straightforward to find the quick-fix setting of commit.status-check.num-retries SEE [iceberg configuration](https://iceberg.apache.org/docs/latest/configuration/)
   
   For anyone else seeing this or testing, if you have a high frequency write scenario you can increase the commit retries by modifying your existing table via an alter table statement, or including the property in initial create table statement. I personally use dataframe v2 for create and append.
   
   ```python
   spark.sql('''
   ALTER TABLE iceberg.performance_tests.parallel_test2 SET TBLPROPERTIES (
     'commit.retry.num-retries'='200'
   )
   ''').show()
   ```
   
   Then make the commit.retry.num-retries change on append (or include it on the create table from the start) if you are using dataframe v2 write api
   ```python   
   sample_spark_df.sortWithinPartitions(col('timestamp'), col('group_id')) \
                               .writeTo('iceberg.performance_tests.parallel_test2') \
                               .tableProperty('write.data.path', 's3a://iceberg_tests/orc_parallel_test2') \
                               .tableProperty('write.format.default', 'orc') \
                               .tableProperty('write.orc.compression-codec', 'zstd') \
                               .tableProperty('write.spark.fanout.enabled', 'True') \
                               .tableProperty('write.target-file-size-bytes', '268435456') \
                               .tableProperty('commit.retry.num-retries', '200') \
                               .partitionedBy(days(col('timestamp')), col('group_id')) \
                               .append()
   ```                            
   
   Ultimately I was able to ensure success 100% of the time with 30x50 (1500 table-write/commit test) but my insert times per 50 tables did drop from single job (50 sequential writes) from 8mins to roughly 45mins when launching all 30 job attempts in parallel. I would love any additional recommendations on improving high-frequency write performance, but do understand there is a tradeoff required in this respect, to achieve ACID compliance and maintain snapshot-insert order. 
   
   It does still seem to me that hive locking can be improved significantly and I will closely follow [2547](https://github.com/apache/iceberg/pull/2547)
   


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