[
https://issues.apache.org/jira/browse/NIFI-11987?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Peter Turcsanyi updated NIFI-11987:
-----------------------------------
Description:
NIFI-11758 introduced FileResourceService used by PutAzureBlobStorage_v12 for
loading the input file directly from disk (skipping content repository).
In case of direct file access, the Azure client lib uses
{{java.nio.FileChannel}} with {{MappedByteBuffer}}-s allocated in the native
memory. The [default buffer size is 4096
bytes|https://github.com/Azure/azure-sdk-for-java/blob/eedfd0ad9db9ee38a0f12943de647267d0649396/sdk/core/azure-core/src/main/java/com/azure/core/util/FluxUtil.java#L257-L259]
which is too small for large files. The library loads the first 256 MB (by
default) of the input file in order to decide single vs multiple upload. It
means 256 MB / 4K = 65536 chunks in native memory which may exceed
{{vm.max_map_count}} limit on Linux (e.g. it is 65530 by default on RedHat 8
and Ubuntu 22.04). The result is OutOfMemoryError:
{code:java}
2023-08-21 16:12:36,751 ERROR
org.apache.nifi.processors.azure.storage.PutAzureBlobStorage_v12:
PutAzureBlobStorage_v12[id=f8fe3228-0189-1000-0000-00003f4dc44d] Failed to
create blob on Azure Blob Storage
reactor.core.Exceptions$ReactiveException: java.io.IOException: Map failed
at reactor.core.Exceptions.propagate(Exceptions.java:396)
at
reactor.core.publisher.BlockingSingleSubscriber.blockingGet(BlockingSingleSubscriber.java:97)
at reactor.core.publisher.Mono.block(Mono.java:1742)
at
com.azure.storage.common.implementation.StorageImplUtils.blockWithOptionalTimeout(StorageImplUtils.java:129)
at
com.azure.storage.blob.BlobClient.uploadWithResponse(BlobClient.java:337)
at
org.apache.nifi.processors.azure.storage.PutAzureBlobStorage_v12.onTrigger(PutAzureBlobStorage_v12.java:204)
at
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
at
org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1360)
at
org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:246)
at
org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:102)
at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
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:750)
Suppressed: java.lang.Exception: #block terminated with an error
at
reactor.core.publisher.BlockingSingleSubscriber.blockingGet(BlockingSingleSubscriber.java:99)
... 16 common frames omitted
Caused by: java.io.IOException: Map failed
at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:938)
at
com.azure.core.util.FluxUtil.lambda$toFluxByteBuffer$4(FluxUtil.java:276)
at
reactor.core.publisher.FluxGenerate$GenerateSubscription.slowPath(FluxGenerate.java:271)
at
reactor.core.publisher.FluxGenerate$GenerateSubscription.request(FluxGenerate.java:213)
at
reactor.core.publisher.FluxFilterFuseable$FilterFuseableSubscriber.request(FluxFilterFuseable.java:191)
at
reactor.core.publisher.FluxConcatMapNoPrefetch$FluxConcatMapNoPrefetchSubscriber.request(FluxConcatMapNoPrefetch.java:336)
at
reactor.core.publisher.FluxSwitchOnFirst$AbstractSwitchOnFirstMain.onSubscribe(FluxSwitchOnFirst.java:499)
at
reactor.core.publisher.FluxConcatMapNoPrefetch$FluxConcatMapNoPrefetchSubscriber.onSubscribe(FluxConcatMapNoPrefetch.java:164)
at
reactor.core.publisher.FluxFilterFuseable$FilterFuseableSubscriber.onSubscribe(FluxFilterFuseable.java:87)
at reactor.core.publisher.FluxGenerate.subscribe(FluxGenerate.java:85)
at reactor.core.publisher.Mono.subscribe(Mono.java:4490)
at reactor.core.publisher.Mono.block(Mono.java:1741)
... 15 common frames omitted
Caused by: java.lang.OutOfMemoryError: Map failed
at sun.nio.ch.FileChannelImpl.map0(Native Method)
at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:935)
... 26 common frames omitted
{code}
The OS limit can be increased but as a more robust solution, we should skip the
default read buffer size and configure a larger value in order to decrease the
count of {{MappedByteBuffer}}-s and native memory allocations.
Suggested size: 4 MB buffer which is the same used by the Azure client lib too
when {{BlobParallelUploadOptions}} is initialized with
[InputStream|https://github.com/Azure/azure-sdk-for-java/blob/eedfd0ad9db9ee38a0f12943de647267d0649396/sdk/storage/azure-storage-blob/src/main/java/com/azure/storage/blob/options/BlobParallelUploadOptions.java#L78]
(coming from the default {{ParallelTransferOptions}}
[here|https://github.com/Azure/azure-sdk-for-java/blob/658dab908e36cee6818288c5708b3345a99c3f86/sdk/storage/azure-storage-blob/src/main/java/com/azure/storage/blob/BlobAsyncClient.java#L708-L709]).
was:
NIFI-11758 introduced FileResourceService used by PutAzureBlobStorage_v12 for
loading the input file directly from disk (skipping content repository).
In case of direct file access, the Azure client lib uses
{{java.nio.FileChannel}} with {{MappedByteBuffer}}-s allocated in the native
memory. The [default buffer size is 4096
bytes|https://github.com/Azure/azure-sdk-for-java/blob/eedfd0ad9db9ee38a0f12943de647267d0649396/sdk/core/azure-core/src/main/java/com/azure/core/util/FluxUtil.java#L257-L259]
which is too small for large files. The library loads the first 256 MB (by
default) of the input file in order to decide single vs multiple upload. It
means 256 MB / 4K = 65536 chunks in native memory which may exceed
{{vm.max_map_count}} limit on Linux (e.g. it is 65530 by default on RedHat 8
and Ubuntu 22.04). The result is OutOfMemoryError:
{code:java}
2023-08-21 16:12:36,751 ERROR
org.apache.nifi.processors.azure.storage.PutAzureBlobStorage_v12:
PutAzureBlobStorage_v12[id=f8fe3228-0189-1000-0000-00003f4dc44d] Failed to
create blob on Azure Blob Storage
reactor.core.Exceptions$ReactiveException: java.io.IOException: Map failed
at reactor.core.Exceptions.propagate(Exceptions.java:396)
at
reactor.core.publisher.BlockingSingleSubscriber.blockingGet(BlockingSingleSubscriber.java:97)
at reactor.core.publisher.Mono.block(Mono.java:1742)
at
com.azure.storage.common.implementation.StorageImplUtils.blockWithOptionalTimeout(StorageImplUtils.java:129)
at
com.azure.storage.blob.BlobClient.uploadWithResponse(BlobClient.java:337)
at
org.apache.nifi.processors.azure.storage.PutAzureBlobStorage_v12.onTrigger(PutAzureBlobStorage_v12.java:204)
at
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
at
org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1360)
at
org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:246)
at
org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:102)
at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
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:750)
Suppressed: java.lang.Exception: #block terminated with an error
at
reactor.core.publisher.BlockingSingleSubscriber.blockingGet(BlockingSingleSubscriber.java:99)
... 16 common frames omitted
Caused by: java.io.IOException: Map failed
at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:938)
at
com.azure.core.util.FluxUtil.lambda$toFluxByteBuffer$4(FluxUtil.java:276)
at
reactor.core.publisher.FluxGenerate$GenerateSubscription.slowPath(FluxGenerate.java:271)
at
reactor.core.publisher.FluxGenerate$GenerateSubscription.request(FluxGenerate.java:213)
at
reactor.core.publisher.FluxFilterFuseable$FilterFuseableSubscriber.request(FluxFilterFuseable.java:191)
at
reactor.core.publisher.FluxConcatMapNoPrefetch$FluxConcatMapNoPrefetchSubscriber.request(FluxConcatMapNoPrefetch.java:336)
at
reactor.core.publisher.FluxSwitchOnFirst$AbstractSwitchOnFirstMain.onSubscribe(FluxSwitchOnFirst.java:499)
at
reactor.core.publisher.FluxConcatMapNoPrefetch$FluxConcatMapNoPrefetchSubscriber.onSubscribe(FluxConcatMapNoPrefetch.java:164)
at
reactor.core.publisher.FluxFilterFuseable$FilterFuseableSubscriber.onSubscribe(FluxFilterFuseable.java:87)
at reactor.core.publisher.FluxGenerate.subscribe(FluxGenerate.java:85)
at reactor.core.publisher.Mono.subscribe(Mono.java:4490)
at reactor.core.publisher.Mono.block(Mono.java:1741)
... 15 common frames omitted
Caused by: java.lang.OutOfMemoryError: Map failed
at sun.nio.ch.FileChannelImpl.map0(Native Method)
at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:935)
... 26 common frames omitted
{code}
The OS limit can be increased but as a more robust solution, we should skip the
default read buffer size and configure a larger value in order to decrease the
count of {{MappedByteBuffer}}-s and native memory allocations.
> PutAzureBlobStorage_v12 using FileResourceService may fail with
> OutOfMemoryError
> --------------------------------------------------------------------------------
>
> Key: NIFI-11987
> URL: https://issues.apache.org/jira/browse/NIFI-11987
> Project: Apache NiFi
> Issue Type: Bug
> Reporter: Peter Turcsanyi
> Assignee: Peter Turcsanyi
> Priority: Major
>
> NIFI-11758 introduced FileResourceService used by PutAzureBlobStorage_v12 for
> loading the input file directly from disk (skipping content repository).
> In case of direct file access, the Azure client lib uses
> {{java.nio.FileChannel}} with {{MappedByteBuffer}}-s allocated in the native
> memory. The [default buffer size is 4096
> bytes|https://github.com/Azure/azure-sdk-for-java/blob/eedfd0ad9db9ee38a0f12943de647267d0649396/sdk/core/azure-core/src/main/java/com/azure/core/util/FluxUtil.java#L257-L259]
> which is too small for large files. The library loads the first 256 MB (by
> default) of the input file in order to decide single vs multiple upload. It
> means 256 MB / 4K = 65536 chunks in native memory which may exceed
> {{vm.max_map_count}} limit on Linux (e.g. it is 65530 by default on RedHat 8
> and Ubuntu 22.04). The result is OutOfMemoryError:
> {code:java}
> 2023-08-21 16:12:36,751 ERROR
> org.apache.nifi.processors.azure.storage.PutAzureBlobStorage_v12:
> PutAzureBlobStorage_v12[id=f8fe3228-0189-1000-0000-00003f4dc44d] Failed to
> create blob on Azure Blob Storage
> reactor.core.Exceptions$ReactiveException: java.io.IOException: Map failed
> at reactor.core.Exceptions.propagate(Exceptions.java:396)
> at
> reactor.core.publisher.BlockingSingleSubscriber.blockingGet(BlockingSingleSubscriber.java:97)
> at reactor.core.publisher.Mono.block(Mono.java:1742)
> at
> com.azure.storage.common.implementation.StorageImplUtils.blockWithOptionalTimeout(StorageImplUtils.java:129)
> at
> com.azure.storage.blob.BlobClient.uploadWithResponse(BlobClient.java:337)
> at
> org.apache.nifi.processors.azure.storage.PutAzureBlobStorage_v12.onTrigger(PutAzureBlobStorage_v12.java:204)
> at
> org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
> at
> org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1360)
> at
> org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:246)
> at
> org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:102)
> at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
> 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:750)
> Suppressed: java.lang.Exception: #block terminated with an error
> at
> reactor.core.publisher.BlockingSingleSubscriber.blockingGet(BlockingSingleSubscriber.java:99)
> ... 16 common frames omitted
> Caused by: java.io.IOException: Map failed
> at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:938)
> at
> com.azure.core.util.FluxUtil.lambda$toFluxByteBuffer$4(FluxUtil.java:276)
> at
> reactor.core.publisher.FluxGenerate$GenerateSubscription.slowPath(FluxGenerate.java:271)
> at
> reactor.core.publisher.FluxGenerate$GenerateSubscription.request(FluxGenerate.java:213)
> at
> reactor.core.publisher.FluxFilterFuseable$FilterFuseableSubscriber.request(FluxFilterFuseable.java:191)
> at
> reactor.core.publisher.FluxConcatMapNoPrefetch$FluxConcatMapNoPrefetchSubscriber.request(FluxConcatMapNoPrefetch.java:336)
> at
> reactor.core.publisher.FluxSwitchOnFirst$AbstractSwitchOnFirstMain.onSubscribe(FluxSwitchOnFirst.java:499)
> at
> reactor.core.publisher.FluxConcatMapNoPrefetch$FluxConcatMapNoPrefetchSubscriber.onSubscribe(FluxConcatMapNoPrefetch.java:164)
> at
> reactor.core.publisher.FluxFilterFuseable$FilterFuseableSubscriber.onSubscribe(FluxFilterFuseable.java:87)
> at reactor.core.publisher.FluxGenerate.subscribe(FluxGenerate.java:85)
> at reactor.core.publisher.Mono.subscribe(Mono.java:4490)
> at reactor.core.publisher.Mono.block(Mono.java:1741)
> ... 15 common frames omitted
> Caused by: java.lang.OutOfMemoryError: Map failed
> at sun.nio.ch.FileChannelImpl.map0(Native Method)
> at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:935)
> ... 26 common frames omitted
> {code}
> The OS limit can be increased but as a more robust solution, we should skip
> the default read buffer size and configure a larger value in order to
> decrease the count of {{MappedByteBuffer}}-s and native memory allocations.
> Suggested size: 4 MB buffer which is the same used by the Azure client lib
> too when {{BlobParallelUploadOptions}} is initialized with
> [InputStream|https://github.com/Azure/azure-sdk-for-java/blob/eedfd0ad9db9ee38a0f12943de647267d0649396/sdk/storage/azure-storage-blob/src/main/java/com/azure/storage/blob/options/BlobParallelUploadOptions.java#L78]
> (coming from the default {{ParallelTransferOptions}}
> [here|https://github.com/Azure/azure-sdk-for-java/blob/658dab908e36cee6818288c5708b3345a99c3f86/sdk/storage/azure-storage-blob/src/main/java/com/azure/storage/blob/BlobAsyncClient.java#L708-L709]).
--
This message was sent by Atlassian Jira
(v8.20.10#820010)