[jira] [Commented] (HADOOP-18707) Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2

2023-06-01 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-18707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17728396#comment-17728396
 ] 

Steve Loughran commented on HADOOP-18707:
-

[~nphung] can you get the full logs from that storediag command  on the config 
which doesn't work, executed with the "-w" command to force a write.

also, out of curiosity, if you have an AWS s3 bucket, could you do the same 
with an s3a URL.

it may be some aspect of buffering through windows which should force us to 
switch to array in such a case and hope the user isn't generating so much data 
the JVM heap fills up

> Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2
> --
>
> Key: HADOOP-18707
> URL: https://issues.apache.org/jira/browse/HADOOP-18707
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/azure
>Affects Versions: 3.3.2, 3.3.5, 3.3.4
>Reporter: Nicolas PHUNG
>Priority: Major
> Fix For: 3.3.4
>
>
> Hello,
> I have an issue with Spark 3.3.2 & Spark 3.4.0 to write into Azure Data Lake 
> Storage Gen2 (abfs/abfss scheme). I've got the following errors:
> {code:java}
> warn 13:12:47.554: StdErr from Kernel Process 23/04/19 13:12:47 ERROR 
> FileFormatWriter: Aborting job 
> 6a75949c-1473-4445-b8ab-d125be3f0f21.org.apache.spark.SparkException: Job 
> aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent 
> failure: Lost task 1.0 in stage 0.0 (TID 1) (myhost executor driver): 
> org.apache.hadoop.util.DiskChecker$DiskErrorException: Could not find any 
> valid local directory for datablock-0001-    at 
> org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:462)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301)
>     at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195)    at 
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175)    at 
> org.apache.parquet.hadoop.util.HadoopOutputFile.create(HadoopOutputFile.java:74)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:347)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:314)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:480)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:420)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:409)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.(ParquetOutputWriter.scala:36)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetUtils$$anon$1.newInstance(ParquetUtils.scala:490)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:161)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.(FileFormatDataWriter.scala:146)
>     at 
> org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:389)
>     at 
> org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$1(WriteFiles.scala:100)
>     at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:888)    
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:888)
>     at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)    
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)    at 
> org.apache.spark.rdd.RDD.iterator(RDD.scala:328)    at 
> org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)    at 
> org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)    
> at org.apache.spark.scheduler.Task.run(Task.scala:139)    at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554)
>     at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529)    
> at org.apache.spark.executor.Executor$TaskRun

[jira] [Commented] (HADOOP-18707) Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2

2023-06-01 Thread Anton Ippolitov (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-18707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17728367#comment-17728367
 ] 

Anton Ippolitov commented on HADOOP-18707:
--

FYI this also caused issues in Flink 1.17. Setting 
"fs.azure.data.blocks.buffer" to "array" solved the problem as well.

https://issues.apache.org/jira/browse/FLINK-32241

> Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2
> --
>
> Key: HADOOP-18707
> URL: https://issues.apache.org/jira/browse/HADOOP-18707
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/azure
>Affects Versions: 3.3.2, 3.3.5, 3.3.4
>Reporter: Nicolas PHUNG
>Priority: Major
> Fix For: 3.3.4
>
>
> Hello,
> I have an issue with Spark 3.3.2 & Spark 3.4.0 to write into Azure Data Lake 
> Storage Gen2 (abfs/abfss scheme). I've got the following errors:
> {code:java}
> warn 13:12:47.554: StdErr from Kernel Process 23/04/19 13:12:47 ERROR 
> FileFormatWriter: Aborting job 
> 6a75949c-1473-4445-b8ab-d125be3f0f21.org.apache.spark.SparkException: Job 
> aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent 
> failure: Lost task 1.0 in stage 0.0 (TID 1) (myhost executor driver): 
> org.apache.hadoop.util.DiskChecker$DiskErrorException: Could not find any 
> valid local directory for datablock-0001-    at 
> org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:462)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301)
>     at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195)    at 
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175)    at 
> org.apache.parquet.hadoop.util.HadoopOutputFile.create(HadoopOutputFile.java:74)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:347)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:314)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:480)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:420)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:409)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.(ParquetOutputWriter.scala:36)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetUtils$$anon$1.newInstance(ParquetUtils.scala:490)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:161)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.(FileFormatDataWriter.scala:146)
>     at 
> org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:389)
>     at 
> org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$1(WriteFiles.scala:100)
>     at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:888)    
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:888)
>     at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)    
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)    at 
> org.apache.spark.rdd.RDD.iterator(RDD.scala:328)    at 
> org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)    at 
> org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)    
> at org.apache.spark.scheduler.Task.run(Task.scala:139)    at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554)
>     at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529)    
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:557)    
> 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.j

[jira] [Commented] (HADOOP-18707) Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2

2023-05-31 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-18707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17728118#comment-17728118
 ] 

Steve Loughran commented on HADOOP-18707:
-

good to hear this is working.
# could you also try settings hadoop.tmp.dir  to something else just to see if 
that makes it go away too?
#. storediag should actually attempt to create a file in the temp dir. did that 
work?

> Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2
> --
>
> Key: HADOOP-18707
> URL: https://issues.apache.org/jira/browse/HADOOP-18707
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/azure
>Affects Versions: 3.3.2, 3.3.5, 3.3.4
>Reporter: Nicolas PHUNG
>Priority: Major
> Fix For: 3.3.4
>
>
> Hello,
> I have an issue with Spark 3.3.2 & Spark 3.4.0 to write into Azure Data Lake 
> Storage Gen2 (abfs/abfss scheme). I've got the following errors:
> {code:java}
> warn 13:12:47.554: StdErr from Kernel Process 23/04/19 13:12:47 ERROR 
> FileFormatWriter: Aborting job 
> 6a75949c-1473-4445-b8ab-d125be3f0f21.org.apache.spark.SparkException: Job 
> aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent 
> failure: Lost task 1.0 in stage 0.0 (TID 1) (myhost executor driver): 
> org.apache.hadoop.util.DiskChecker$DiskErrorException: Could not find any 
> valid local directory for datablock-0001-    at 
> org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:462)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301)
>     at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195)    at 
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175)    at 
> org.apache.parquet.hadoop.util.HadoopOutputFile.create(HadoopOutputFile.java:74)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:347)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:314)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:480)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:420)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:409)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.(ParquetOutputWriter.scala:36)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetUtils$$anon$1.newInstance(ParquetUtils.scala:490)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:161)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.(FileFormatDataWriter.scala:146)
>     at 
> org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:389)
>     at 
> org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$1(WriteFiles.scala:100)
>     at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:888)    
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:888)
>     at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)    
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)    at 
> org.apache.spark.rdd.RDD.iterator(RDD.scala:328)    at 
> org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)    at 
> org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)    
> at org.apache.spark.scheduler.Task.run(Task.scala:139)    at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554)
>     at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529)    
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:557)    
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecuto

[jira] [Commented] (HADOOP-18707) Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2

2023-05-26 Thread Nicolas PHUNG (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-18707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1772#comment-1772
 ] 

Nicolas PHUNG commented on HADOOP-18707:


Hello [~ste...@apache.org] 
 * I have tried to use [https://github.com/steveloughran/cloudstore], run its 
"storediag" as you mention.

{code:java}
Hadoop Options
==[001]  fs.defaultFS = "file:///" [core-default.xml]
2023-05-26 17:32:10,911 INFO Configuration.deprecation: fs.default.name is 
deprecated. Instead, use fs.defaultFS
[002]  fs.default.name = "file:///"
[003]  fs.creation.parallel.count = (unset)
[004]  fs.permissions.umask-mode = "022" [core-default.xml]
[005]  fs.trash.classname = (unset)
[006]  fs.trash.interval = "0" [core-default.xml]
[007]  fs.trash.checkpoint.interval = "0" [core-default.xml]
[008]  hadoop.tmp.dir = "/tmp/hadoop" [core-default.xml] {code}
On windows _/tmp_ , I have the following:

__ 
{code:java}
❯ df /tmp -h
Filesystem Size Used Avail Use% Mounted on
E:/my_user/AppData/Local/Temp 236G 161G 75G 69% /tmp{code}
__

I have tried to create the folder and the folder with abfs but no luck on this 
side.

* Setting "fs.azure.data.blocks.buffer" to "array" to go back to memory is 
working for me. So that I have the same behavior as on previous version of 
Spark 3.1.2 with hadoop-azure 3.2. I've tried successfully with *Spark 3.4.0 
with hadoop-azure 3.3.4*. I haven't managed to get *Spark 3.4.0 with 
hadoop-azure 3.3.5*, I'm running into other jars issues between Hadoop and 
Spark.

This solves my issue for local setup. Thanks for your help.

> Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2
> --
>
> Key: HADOOP-18707
> URL: https://issues.apache.org/jira/browse/HADOOP-18707
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/azure
>Affects Versions: 3.3.2, 3.3.5, 3.3.4
>Reporter: Nicolas PHUNG
>Priority: Major
>
> Hello,
> I have an issue with Spark 3.3.2 & Spark 3.4.0 to write into Azure Data Lake 
> Storage Gen2 (abfs/abfss scheme). I've got the following errors:
> {code:java}
> warn 13:12:47.554: StdErr from Kernel Process 23/04/19 13:12:47 ERROR 
> FileFormatWriter: Aborting job 
> 6a75949c-1473-4445-b8ab-d125be3f0f21.org.apache.spark.SparkException: Job 
> aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent 
> failure: Lost task 1.0 in stage 0.0 (TID 1) (myhost executor driver): 
> org.apache.hadoop.util.DiskChecker$DiskErrorException: Could not find any 
> valid local directory for datablock-0001-    at 
> org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:462)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301)
>     at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195)    at 
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175)    at 
> org.apache.parquet.hadoop.util.HadoopOutputFile.create(HadoopOutputFile.java:74)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:347)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:314)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:480)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:420)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:409)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.(ParquetOutputWriter.scala:36)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetUtils$$anon$1.newInstance(ParquetUtils.scala:490)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:161)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.(FileFormatDataWriter.scala:146)
>     at 
> org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:389)
>

[jira] [Commented] (HADOOP-18707) Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2

2023-04-28 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-18707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17717615#comment-17717615
 ] 

Steve Loughran commented on HADOOP-18707:
-

on hadoop azure 3.3.2+ we default to using HDD for buffering, before that it 
was using heap, and on a process with many threads, would run out of heap 
space. So there is disk use which never took place before, approximately 
worker-threads * block size * blocks-per-output-stream. Sounds more likely to 
be config here though

> Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2
> --
>
> Key: HADOOP-18707
> URL: https://issues.apache.org/jira/browse/HADOOP-18707
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/azure
>Affects Versions: 3.3.2, 3.3.5, 3.3.4
>Reporter: Nicolas PHUNG
>Priority: Major
>
> Hello,
> I have an issue with Spark 3.3.2 & Spark 3.4.0 to write into Azure Data Lake 
> Storage Gen2 (abfs/abfss scheme). I've got the following errors:
> {code:java}
> warn 13:12:47.554: StdErr from Kernel Process 23/04/19 13:12:47 ERROR 
> FileFormatWriter: Aborting job 
> 6a75949c-1473-4445-b8ab-d125be3f0f21.org.apache.spark.SparkException: Job 
> aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent 
> failure: Lost task 1.0 in stage 0.0 (TID 1) (myhost executor driver): 
> org.apache.hadoop.util.DiskChecker$DiskErrorException: Could not find any 
> valid local directory for datablock-0001-    at 
> org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:462)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301)
>     at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195)    at 
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175)    at 
> org.apache.parquet.hadoop.util.HadoopOutputFile.create(HadoopOutputFile.java:74)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:347)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:314)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:480)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:420)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:409)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.(ParquetOutputWriter.scala:36)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetUtils$$anon$1.newInstance(ParquetUtils.scala:490)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:161)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.(FileFormatDataWriter.scala:146)
>     at 
> org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:389)
>     at 
> org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$1(WriteFiles.scala:100)
>     at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:888)    
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:888)
>     at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)    
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)    at 
> org.apache.spark.rdd.RDD.iterator(RDD.scala:328)    at 
> org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)    at 
> org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)    
> at org.apache.spark.scheduler.Task.run(Task.scala:139)    at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554)
>     at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529)    
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:557)    
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>     a

[jira] [Commented] (HADOOP-18707) Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2

2023-04-27 Thread Nicolas PHUNG (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-18707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17717481#comment-17717481
 ] 

Nicolas PHUNG commented on HADOOP-18707:


Thanks for your answer [~ste...@apache.org] 

I'll try to take a look today.

I forgot to mention that the spark job is running from a local disk and the 
write is to abfs/abfss scheme on Azure Data Lake Storage Gen2. I don't think 
it's an issue with disk space at least. Because when I'm using the same code, I 
can write as expected with Spark 3.1.2 with hadoop-azure 3.2.1.

> Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2
> --
>
> Key: HADOOP-18707
> URL: https://issues.apache.org/jira/browse/HADOOP-18707
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/azure
>Affects Versions: 3.3.2, 3.3.5, 3.3.4
>Reporter: Nicolas PHUNG
>Priority: Major
>
> Hello,
> I have an issue with Spark 3.3.2 & Spark 3.4.0 to write into Azure Data Lake 
> Storage Gen2 (abfs/abfss scheme). I've got the following errors:
> {code:java}
> warn 13:12:47.554: StdErr from Kernel Process 23/04/19 13:12:47 ERROR 
> FileFormatWriter: Aborting job 
> 6a75949c-1473-4445-b8ab-d125be3f0f21.org.apache.spark.SparkException: Job 
> aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent 
> failure: Lost task 1.0 in stage 0.0 (TID 1) (myhost executor driver): 
> org.apache.hadoop.util.DiskChecker$DiskErrorException: Could not find any 
> valid local directory for datablock-0001-    at 
> org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:462)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301)
>     at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195)    at 
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175)    at 
> org.apache.parquet.hadoop.util.HadoopOutputFile.create(HadoopOutputFile.java:74)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:347)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:314)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:480)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:420)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:409)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.(ParquetOutputWriter.scala:36)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetUtils$$anon$1.newInstance(ParquetUtils.scala:490)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:161)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.(FileFormatDataWriter.scala:146)
>     at 
> org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:389)
>     at 
> org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$1(WriteFiles.scala:100)
>     at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:888)    
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:888)
>     at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)    
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)    at 
> org.apache.spark.rdd.RDD.iterator(RDD.scala:328)    at 
> org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)    at 
> org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)    
> at org.apache.spark.scheduler.Task.run(Task.scala:139)    at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554)
>     at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529)    
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:557)    
> at 
> java.util.concurrent.ThreadPoolExecutor

[jira] [Commented] (HADOOP-18707) Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2

2023-04-27 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/HADOOP-18707?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17717093#comment-17717093
 ] 

Steve Loughran commented on HADOOP-18707:
-

failure to find temp dir to buffer output, "Could not find any valid local 
directory for datablock-0001-"; which comes from "fs.azure.buffer.dir", default 
of ${hadoop.tmp.dir}/abfs

cause would be HADOOP-17195 and buffering blocks via HDD rather than memory. so 
either you've not got any temp dir for it, or have run out of space

* download https://github.com/steveloughran/cloudstore, run its "storediag" 
command against your bucket, with a core-site.xml (or supplied xml file) to get 
printout of values and some probes of it
* you can switch to memory storage by setting  "fs.azure.data.blocks.buffer" to 
"array" to go back to memory (and risk of running out, though now you can 
control the amount used more carefully)

> Cannot write to Azure Datalake Gen2 (abfs/abfss) after Spark 3.1.2
> --
>
> Key: HADOOP-18707
> URL: https://issues.apache.org/jira/browse/HADOOP-18707
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/azure
>Affects Versions: 3.3.2, 3.3.5, 3.3.4
>Reporter: Nicolas PHUNG
>Priority: Major
>
> Hello,
> I have an issue with Spark 3.3.2 & Spark 3.4.0 to write into Azure Data Lake 
> Storage Gen2 (abfs/abfss scheme). I've got the following errors:
> {code:java}
> warn 13:12:47.554: StdErr from Kernel Process 23/04/19 13:12:47 ERROR 
> FileFormatWriter: Aborting job 
> 6a75949c-1473-4445-b8ab-d125be3f0f21.org.apache.spark.SparkException: Job 
> aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent 
> failure: Lost task 1.0 in stage 0.0 (TID 1) (myhost executor driver): 
> org.apache.hadoop.util.DiskChecker$DiskErrorException: Could not find any 
> valid local directory for datablock-0001-    at 
> org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:462)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:165)
>     at 
> org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:146)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.createTmpFileForWrite(DataBlocks.java:980)
>     at 
> org.apache.hadoop.fs.store.DataBlocks$DiskBlockFactory.create(DataBlocks.java:960)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.createBlockIfNeeded(AbfsOutputStream.java:262)
>     at 
> org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream.(AbfsOutputStream.java:173)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.createFile(AzureBlobFileSystemStore.java:580)
>     at 
> org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.create(AzureBlobFileSystem.java:301)
>     at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195)    at 
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175)    at 
> org.apache.parquet.hadoop.util.HadoopOutputFile.create(HadoopOutputFile.java:74)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:347)
>     at 
> org.apache.parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:314)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:480)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:420)
>     at 
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:409)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.(ParquetOutputWriter.scala:36)
>     at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetUtils$$anon$1.newInstance(ParquetUtils.scala:490)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:161)
>     at 
> org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.(FileFormatDataWriter.scala:146)
>     at 
> org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:389)
>     at 
> org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$1(WriteFiles.scala:100)
>     at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:888)    
> at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:888)
>     at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)    
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)    at 
> org.apache.spark.rdd.RDD.iterator(RDD.scala:328)    at 
> org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)    at 
> org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)