You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/01/10 20:38:18 UTC

[GitHub] [hudi] vinothchandar opened a new issue #2429: [SUPPORT] S3 throws ConnectionPoolTimeoutException: Timeout waiting for connection from pool when metadata table is turned on

vinothchandar opened a new issue #2429:
URL: https://github.com/apache/hudi/issues/2429


   **Describe the problem you faced**
   
   This is only on master branch, with the upcoming 0.7.0. 
   
   When turning on metadata table using `hoodie.metadata.enable=true`, we see a hang. 
   
   ![hadoop3-timeout-emr](https://user-images.githubusercontent.com/1179324/104134708-4674bc80-5340-11eb-9149-ba8032d7beb3.png)
   
   Normally, its very quick. 
   
   ```
   Writing round 2/20
   2021-01-10 17:23:08,734 WARN scheduler.TaskSetManager: Lost task 4.0 in stage 66.0 (TID 178, , executor 1): 
   org.apache.hudi.exception.HoodieIOException: Failed to list base files in partition date-96
           at org.apache.hudi.common.table.view.AbstractTableFileSystemView.lambda$ensurePartitionLoadedCorrectly$9(AbstractTableFileSystemView.java:290)
           at java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1660)
           at org.apache.hudi.common.table.view.AbstractTableFileSystemView.ensurePartitionLoadedCorrectly(AbstractTableFileSystemView.java:269)
           at org.apache.hudi.common.table.view.AbstractTableFileSystemView.getLatestBaseFilesBeforeOrOn(AbstractTableFileSystemView.java:455)
           at org.apache.hudi.table.action.commit.UpsertPartitioner.getSmallFiles(UpsertPartitioner.java:277)
           at org.apache.hudi.table.action.commit.UpsertPartitioner.lambda$getSmallFilesForPartitions$f1d92f9e$1(UpsertPartitioner.java:258)
           at org.apache.spark.api.java.JavaPairRDD$.$anonfun$pairFunToScalaFun$1(JavaPairRDD.scala:1044)
           at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
           at scala.collection.Iterator.foreach(Iterator.scala:941)
           at scala.collection.Iterator.foreach$(Iterator.scala:941)
           at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
           at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:62)
           at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:53)
           at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:105)
           at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:49)
           at scala.collection.TraversableOnce.to(TraversableOnce.scala:315)
           at scala.collection.TraversableOnce.to$(TraversableOnce.scala:313)
           at scala.collection.AbstractIterator.to(Iterator.scala:1429)
           at scala.collection.TraversableOnce.toBuffer(TraversableOnce.scala:307)
           at scala.collection.TraversableOnce.toBuffer$(TraversableOnce.scala:307)
           at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1429)
           at scala.collection.TraversableOnce.toArray(TraversableOnce.scala:294)
           at scala.collection.TraversableOnce.toArray$(TraversableOnce.scala:288)
           at scala.collection.AbstractIterator.toArray(Iterator.scala:1429)
           at org.apache.spark.rdd.RDD.$anonfun$collect$2(RDD.scala:1004)
           at org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2139)
           at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
           at org.apache.spark.scheduler.Task.run(Task.scala:127)
           at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446)
           at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
           at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   Caused by: java.io.InterruptedIOException: getFileStatus on s3a://XXXX/0_7_0_testing/output-cow-2/org.apache.hudi/date-96:
    com.amazonaws.SdkClientException: Unable to execute HTTP request: Timeout waiting for connection from pool
           at org.apache.hadoop.fs.s3a.S3AUtils.translateInterruptedException(S3AUtils.java:352)
           at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:177)
           at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:151)
           at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2201)
           at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:2163)
           at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:2102)
           at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1903)
           at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$listStatus$9(S3AFileSystem.java:1882)
           at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109)
           at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:1882)
           at org.apache.hudi.metadata.BaseTableMetadata.getAllFilesInPartition(BaseTableMetadata.java:129)
           at org.apache.hudi.metadata.HoodieMetadataFileSystemView.listPartition(HoodieMetadataFileSystemView.java:61)
           at org.apache.hudi.common.table.view.AbstractTableFileSystemView.lambda$ensurePartitionLoadedCorrectly$9(AbstractTableFileSystemView.java:280)
           ... 33 more
   Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: Timeout waiting for connection from pool
           at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleRetryableException(AmazonHttpClient.java:1116)
           at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1066)
           at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
           at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
           at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
           at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
           at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
           at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
           at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4368)
           at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4315)
           at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1271)
           at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getObjectMetadata$4(S3AFileSystem.java:1249)
           at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:322)
           at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:285)
           at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1246)
           at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2183)
           ... 42 more
   Caused by: com.amazonaws.thirdparty.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool
           at com.amazonaws.thirdparty.apache.http.impl.conn.PoolingHttpClientConnectionManager.leaseConnection(PoolingHttpClientConnectionManager.java:313)
           at com.amazonaws.thirdparty.apache.http.impl.conn.PoolingHttpClientConnectionManager$1.get(PoolingHttpClientConnectionManager.java:279)
           at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
           at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
           at java.lang.reflect.Method.invoke(Method.java:498)
           at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70)
           at com.amazonaws.http.conn.$Proxy20.get(Unknown Source)
           at com.amazonaws.thirdparty.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:191)
           at com.amazonaws.thirdparty.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:185)
           at com.amazonaws.thirdparty.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185)
           at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83)
           at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:56)
           at com.amazonaws.http.apache.client.impl.SdkHttpClient.execute(SdkHttpClient.java:72)
           at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1238)
           at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1058)
           ... 56 more
   ```
   
   Thread dumping an executor shows 
   
   ```Shell
   java.lang.Thread.sleep(Native Method)
   com.amazonaws.http.AmazonHttpClient$RequestExecutor.doPauseBeforeRetry(AmazonHttpClient.java:1679)
   com.amazonaws.http.AmazonHttpClient$RequestExecutor.pauseBeforeRetry(AmazonHttpClient.java:1653)
   com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1191)
   com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1058)
   com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
   com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
   com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
   com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
   com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
   com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
   com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4368)
   com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4315)
   com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1271)
   org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getObjectMetadata$4(S3AFileSystem.java:1249)
   org.apache.hadoop.fs.s3a.S3AFileSystem$$Lambda$622/148788126.execute(Unknown Source)
   org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:322)
   org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:285)
   org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1246)
   org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2183)
   org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:2163)
   org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:2102)
   org.apache.hudi.common.fs.HoodieWrapperFileSystem.lambda$getFileStatus$17(HoodieWrapperFileSystem.java:393)
   org.apache.hudi.common.fs.HoodieWrapperFileSystem$$Lambda$851/437862677.get(Unknown Source)
   org.apache.hudi.common.fs.HoodieWrapperFileSystem.executeFuncWithTimeMetrics(HoodieWrapperFileSystem.java:100)
   org.apache.hudi.common.fs.HoodieWrapperFileSystem.getFileStatus(HoodieWrapperFileSystem.java:387)
   org.apache.hudi.exception.TableNotFoundException.checkTableValidity(TableNotFoundException.java:47)
   org.apache.hudi.common.table.HoodieTableMetaClient.<init>(HoodieTableMetaClient.java:128)
   org.apache.hudi.common.table.HoodieTableMetaClient.<init>(HoodieTableMetaClient.java:114)
   org.apache.hudi.common.table.HoodieTableMetaClient.<init>(HoodieTableMetaClient.java:100)
   org.apache.hudi.metadata.HoodieBackedTableMetadata.<init>(HoodieBackedTableMetadata.java:85)
   org.apache.hudi.metadata.HoodieTableMetadata.create(HoodieTableMetadata.java:73)
   org.apache.hudi.table.HoodieTable.metadata(HoodieTable.java:663)
   org.apache.hudi.table.HoodieTable.getFileSystemViewInternal(HoodieTable.java:280)
   org.apache.hudi.table.HoodieTable.getBaseFileOnlyView(HoodieTable.java:260)
   org.apache.hudi.table.action.commit.UpsertPartitioner.getSmallFiles(UpsertPartitioner.java:276)
   org.apache.hudi.table.action.commit.UpsertPartitioner.lambda$getSmallFilesForPartitions$f1d92f9e$1(UpsertPartitioner.java:258)
   org.apache.hudi.table.action.commit.UpsertPartitioner$$Lambda$786/813724600.call(Unknown Source)
   ```
   
   **Environment Description**
   
   * Hudi version : 0.6.1-SNAPSHOT
   
   * Spark version : 3.0.1
   
   * Hive version :
   
   * Hadoop version : 3.2.0
   
   * Storage (HDFS/S3/GCS..) : S3
   
   * Running on Docker? (yes/no) :
   
   
   
   


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



[GitHub] [hudi] nsivabalan commented on issue #2429: [SUPPORT] S3 throws ConnectionPoolTimeoutException: Timeout waiting for connection from pool when metadata table is turned on

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #2429:
URL: https://github.com/apache/hudi/issues/2429#issuecomment-757552669


   sure. 
   I ran a long running test-suite job in cluster. w/o metadata listing enabled, things were good. But w/ metadata enabled, ran into too many open files problems after 20 iterations(each iteration has inserts, updates, deletes, spark datasource read) 
   
   ```
   21/01/10 06:42:04 WARN TaskSetManager: Lost task 0.0 in stage 3909.0 (TID 455057, agent6922-abc.com, executor 1): java.io.UncheckedIOException: java.net.SocketException: Too many open files
           at org.apache.hudi.integ.testsuite.generator.DeltaGenerator.lambda$writeRecords$5e8e5895$1(DeltaGenerator.java:123)
           at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$4$1.apply(JavaRDDLike.scala:153)
           at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$4$1.apply(JavaRDDLike.scala:153)
           at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$23.apply(RDD.scala:801)
           at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$23.apply(RDD.scala:801)
           at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
           at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
           at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
           at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
           at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
           at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
           at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
           at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
           at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
           at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
           at org.apache.spark.scheduler.Task.run(Task.scala:123)
           at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
           at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
           at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
                  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   Caused by: java.net.SocketException: Too many open files
           at sun.nio.ch.Net.socket0(Native Method)
           at sun.nio.ch.Net.socket(Net.java:411)
           at sun.nio.ch.Net.socket(Net.java:404)
           at sun.nio.ch.SocketChannelImpl.<init>(SocketChannelImpl.java:105)
           at sun.nio.ch.SelectorProviderImpl.openSocketChannel(SelectorProviderImpl.java:60)
           at java.nio.channels.SocketChannel.open(SocketChannel.java:145)
           at org.apache.hadoop.net.StandardSocketFactory.createSocket(StandardSocketFactory.java:62)
           at org.apache.hadoop.hdfs.DFSOutputStream.createSocketForPipeline(DFSOutputStream.java:1531)
           at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.createBlockOutputStream(DFSOutputStream.java:1309)
           at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.nextBlockOutputStream(DFSOutputStream.java:1262)
           at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:448)
   
   21/01/10 06:42:05 WARN TaskSetManager: Lost task 3.0 in stage 3911.0 (TID 455068, agent6922-abc.com, executor 1): java.io.FileNotFoundException: /opt/hudi/shared/yarn/cache/data/nm/usercache/sivabala/appcache/application_1609962097310_747119/blockmgr-7f5d0680-3597-426f-8a2b-29f7411e130b/0c/temp_shuffle_a70e65a8-3e98-4d5a-a714-ef3c44c2fe4e (Too many open files)
           at java.io.FileOutputStream.open0(Native Method)
           at java.io.FileOutputStream.open(FileOutputStream.java:270)
           at java.io.FileOutputStream.<init>(FileOutputStream.java:213)
           at org.apache.spark.storage.DiskBlockObjectWriter.initialize(DiskBlockObjectWriter.scala:103)
           at org.apache.spark.storage.DiskBlockObjectWriter.open(DiskBlockObjectWriter.scala:116)
           at org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:237)
           at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:151)
               at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
           at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:55)
           at org.apache.spark.scheduler.Task.run(Task.scala:123)
           at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
           at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
           at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   
   21/01/10 06:42:05 WARN TaskSetManager: Lost task 1.0 in stage 3911.0 (TID 455066, agent6922-abc.com, executor 1): java.io.FileNotFoundException: /opt/hudi/shared/yarn/cache/data/nm/usercache/sivabala/appcache/application_1609962097310_747119/blockmgr-7f5d0680-3597-426f-8a2b-29f7411e130b/03/temp_shuffle_f362ca47-2b91-457a-b3ca-33570de418bd (Too many open files)
           at java.io.FileOutputStream.open0(Native Method)
           at java.io.FileOutputStream.open(FileOutputStream.java:270)
           at java.io.FileOutputStream.<init>(FileOutputStream.java:213)
           at org.apache.spark.storage.DiskBlockObjectWriter.initialize(DiskBlockObjectWriter.scala:103)
           at org.apache.spark.storage.DiskBlockObjectWriter.open(DiskBlockObjectWriter.scala:116)
           at org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:237)
           at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:151)
           at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
           at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:55)
           at org.apache.spark.scheduler.Task.run(Task.scala:123)
           at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
           at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
           at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   
   ```
   


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



[GitHub] [hudi] nsivabalan commented on issue #2429: [SUPPORT] S3 throws ConnectionPoolTimeoutException: Timeout waiting for connection from pool when metadata table is turned on

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #2429:
URL: https://github.com/apache/hudi/issues/2429#issuecomment-766428773


   @vinothchandar : closing this for now. feel free to re-open if you see more issues. 


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



[GitHub] [hudi] vinothchandar commented on issue #2429: [SUPPORT] S3 throws ConnectionPoolTimeoutException: Timeout waiting for connection from pool when metadata table is turned on

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on issue #2429:
URL: https://github.com/apache/hudi/issues/2429#issuecomment-757545756


   @nsivabalan can you please add the stacktrace you saw?


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



[GitHub] [hudi] nsivabalan commented on issue #2429: [SUPPORT] S3 throws ConnectionPoolTimeoutException: Timeout waiting for connection from pool when metadata table is turned on

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #2429:
URL: https://github.com/apache/hudi/issues/2429#issuecomment-766428773


   @vinothchandar : closing this for now. feel free to re-open if you see more issues. 


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



[GitHub] [hudi] vinothchandar commented on issue #2429: [SUPPORT] S3 throws ConnectionPoolTimeoutException: Timeout waiting for connection from pool when metadata table is turned on

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on issue #2429:
URL: https://github.com/apache/hudi/issues/2429#issuecomment-757540328


   Just bumping up `sc.hadoopConfiguration.setInt("fs.s3a.connection.maximum", 100)` makes the job go forward more rounds of writing. There is some kind of leak somewhere. 
   
   cc @prashantwason 


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



[GitHub] [hudi] nsivabalan closed issue #2429: [SUPPORT] S3 throws ConnectionPoolTimeoutException: Timeout waiting for connection from pool when metadata table is turned on

Posted by GitBox <gi...@apache.org>.
nsivabalan closed issue #2429:
URL: https://github.com/apache/hudi/issues/2429


   


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



[GitHub] [hudi] nsivabalan closed issue #2429: [SUPPORT] S3 throws ConnectionPoolTimeoutException: Timeout waiting for connection from pool when metadata table is turned on

Posted by GitBox <gi...@apache.org>.
nsivabalan closed issue #2429:
URL: https://github.com/apache/hudi/issues/2429


   


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