You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/03/10 16:12:41 UTC

[GitHub] [arrow] davisusanibar opened a new pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

davisusanibar opened a new pull request #12603:
URL: https://github.com/apache/arrow/pull/12603


   Apache Arrow java memory documentation.
   
   @lidavidm 


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r827444879



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.

Review comment:
       Thanks, changed

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible

Review comment:
       Added




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r826442524



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.
+
+.. code-block:: Java
+
+    try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE) ) { ; }

Review comment:
       Deleted

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.

Review comment:
       Added




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828340164



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.

Review comment:
       Updated




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r829062849



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,208 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block::
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow modules define simple logging facade for java SLF4J, configure it properly to see your logs (e.g. Logback/Log4J).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    import org.apache.arrow.vector.IntVector;
+
+    try (BufferAllocator bufferAllocator = new RootAllocator(Integer.MAX_VALUE)) {
+        final int QUANTITY = 5;
+        try (IntVector intVector = new IntVector("int-01", bufferAllocator)) {
+            intVector.allocateNew(QUANTITY);
+            for (int i = 0; i < QUANTITY; i++) {
+                intVector.set(i, i);
+            }
+            intVector.setValueCount(QUANTITY);
+        }
+        // Fix the next code!, it is only to see the track of allocators when debug is enabled
+        IntVector intVectorV = new IntVector("int-02", bufferAllocator);
+        intVectorV.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV.set(i, i);
+        }
+        intVectorV.setValueCount(QUANTITY);
+
+        BufferAllocator childAllocator = bufferAllocator.newChildAllocator("child-isolated", 0,
+                Integer.MAX_VALUE / 4);
+        IntVector intVectorV2 = new IntVector("int-isolated-01", childAllocator);
+        intVectorV2.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV2.set(i, i);
+        }
+    }

Review comment:
       I'll submit a jira/PR for this later (there's some issues that need fixing for the historical log) but I think things are good here as is. 




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r829065000



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,189 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block:: shell
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with `ReferenceManager.release`_ and
+`ReferenceManager.retain`_.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow logs some allocation information via SLF4J; configure it properly to see these logs (e.g. via Logback/Apache Log4j).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+    }
+
+Logs without debug allocators enabled:
+
+.. code-block:: shell
+
+    11:56:48.944 [main] INFO  o.apache.arrow.memory.BaseAllocator - Debug mode disabled.
+    ArrowBuf[2], address:140508391276544, length:4096
+    16:28:08.847 [main] ERROR o.apache.arrow.memory.BaseAllocator - Memory was leaked by query. Memory leaked: (4096)
+    Allocator(ROOT) 0/4096/4096/8192 (res/actual/peak/limit)
+
+Logs with debug allocators enabled:
+
+.. code-block:: shell
+
+    11:56:48.944 [main] INFO  o.apache.arrow.memory.BaseAllocator - Debug mode enabled.
+    ArrowBuf[2], address:140437894463488, length:4096
+    Exception in thread "main" java.lang.IllegalStateException: Allocator[ROOT] closed with outstanding buffers allocated (1).
+    Allocator(ROOT) 0/4096/4096/8192 (res/actual/peak/limit)
+      child allocators: 0
+      ledgers: 1
+        ledger[1] allocator: ROOT), isOwning: , size: , references: 1, life: 261438177096661..0, allocatorManager: [, life: ] holds 1 buffers.
+            ArrowBuf[2], address:140437894463488, length:4096
+      reservations: 0
+
+.. _`BufferAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/BufferAllocator.html
+.. _`RootAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/RootAllocator.html
+.. _`newChildAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/RootAllocator.html#newChildAllocator-java.lang.String-org.apache.arrow.memory.AllocationListener-long-long-
+.. _`ArrowBuf`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/ArrowBuf.html
+.. _`Reference Counting`: https://github.com/apache/arrow/blob/2092e18752a9c0494799493b12eb1830052217a2/java/memory/memory-core/src/main/java/org/apache/arrow/memory/ReferenceManager.java#L30

Review comment:
       Wait - I don't think we should link into source files. They don't really provide useful context in this tutorial. (That goes for all the links here.)




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r826443178



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.

Review comment:
       Updated




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r827443293



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_

Review comment:
       Deleted




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828342652



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )

Review comment:
       Deleted




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r826442783



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.

Review comment:
       Updated

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.

Review comment:
       Updated

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:

Review comment:
       Updated




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r826443891



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.

Review comment:
       Added an example,  but please @lidavidm  if you could help me with more examples or reason to use child allocators.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] github-actions[bot] commented on pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#issuecomment-1064274834


   https://issues.apache.org/jira/browse/ARROW-15573


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828341681



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).

Review comment:
       Updated




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828469321



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,208 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block::
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow modules define simple logging facade for java SLF4J, configure it properly to see your logs (e.g. Logback/Log4J).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    import org.apache.arrow.vector.IntVector;
+
+    try (BufferAllocator bufferAllocator = new RootAllocator(Integer.MAX_VALUE)) {
+        final int QUANTITY = 5;
+        try (IntVector intVector = new IntVector("int-01", bufferAllocator)) {
+            intVector.allocateNew(QUANTITY);
+            for (int i = 0; i < QUANTITY; i++) {
+                intVector.set(i, i);
+            }
+            intVector.setValueCount(QUANTITY);
+        }
+        // Fix the next code!, it is only to see the track of allocators when debug is enabled
+        IntVector intVectorV = new IntVector("int-02", bufferAllocator);
+        intVectorV.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV.set(i, i);
+        }
+        intVectorV.setValueCount(QUANTITY);
+
+        BufferAllocator childAllocator = bufferAllocator.newChildAllocator("child-isolated", 0,
+                Integer.MAX_VALUE / 4);
+        IntVector intVectorV2 = new IntVector("int-isolated-01", childAllocator);
+        intVectorV2.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV2.set(i, i);
+        }
+    }
+
+.. code-block::
+
+    15:49:32,755 |-INFO in ch.qos.logback.classic.LoggerContext[default] - Found resource [logback-test.xml] at [file:/Users/java/source/demo/target/classes/logback-test.xml]
+    15:49:32,924 |-INFO in ch.qos.logback.classic.joran.action.LoggerAction - Setting level of logger [org.apache.arrow] to DEBUG
+    11:56:48.944 [main] INFO  o.apache.arrow.memory.BaseAllocator - Debug mode enabled.
+    Exception in thread "main" java.lang.IllegalStateException: Allocator[ROOT] closed with outstanding child allocators.
+    Allocator(ROOT) 0/64/64/2147483647 (res/actual/peak/limit)
+      child allocators: 1
+        Allocator(child-isolated) 0/32/32/536870911 (res/actual/peak/limit)
+          child allocators: 0
+          ledgers: 1
+            ledger[3] allocator: child-isolated), isOwning: , size: , references: 2, life: 246918908438818..0, allocatorManager: [, life: ] holds 3 buffers.
+                ArrowBuf[10], address:140408097079352, length:8
+                ArrowBuf[8], address:140408097079328, length:32
+                ArrowBuf[9], address:140408097079328, length:24
+          reservations: 0
+      ledgers: 1
+        ledger[2] allocator: ROOT), isOwning: , size: , references: 2, life: 246tors can be named; this makes it easier to tell where an Arro918906331643..0, allocatorManager: [, life: ] holds 3 buffers.

Review comment:
       Deleted




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828469143



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and

Review comment:
       It was added to ReferenceManager, just added also to release and retain




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r826442304



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.
+
+.. code-block:: Java
+
+    try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE) ) { ; }
+
+ArrowBuf
+========
+
+The facade for interacting directly with a chunk of memory.
+
+Two important instance variables of an ArrowBuf:
+
+* Address: Starting virtual address in the underlying memory chunk that this ArrowBuf has access to.
+* Length: Length (in bytes) in the underlying memory chunk that this ArrowBuf has access to.

Review comment:
       Deleted

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.
+
+.. code-block:: Java
+
+    try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE) ) { ; }
+
+ArrowBuf
+========
+
+The facade for interacting directly with a chunk of memory.

Review comment:
       Added

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.

Review comment:
       Updated

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.

Review comment:
       Added




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r826443285



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.
+
+.. code-block:: Java
+
+    try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE) ) { ; }
+
+ArrowBuf
+========
+
+The facade for interacting directly with a chunk of memory.
+
+Two important instance variables of an ArrowBuf:
+
+* Address: Starting virtual address in the underlying memory chunk that this ArrowBuf has access to.
+* Length: Length (in bytes) in the underlying memory chunk that this ArrowBuf has access to.
+
+Memory Modules
+==============
+
+Memory core define the bases to work with direct memory and the application decided to allocate arrow buffer bases on
+the dependency added on your pom.xml (memory-unsafe or memory-netty). If any of these is not added the application raise
+an exception.

Review comment:
       Updated




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] ursabot edited a comment on pull request #12603: ARROW-15573: [Java][Doc] Document Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#issuecomment-1070939910


   Benchmark runs are scheduled for baseline = 7e70c429ccd67c35651a8891cff5f7983978a70b and contender = 8db93a5328f93255f352d97b7a5a7b01a1368c6e. 8db93a5328f93255f352d97b7a5a7b01a1368c6e is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/d40b6b17b1f942229069705150ad6ec4...a66512ee5f954d5d92977e9f0d7e4572/)
   [Finished :arrow_down:0.29% :arrow_up:0.08%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/670a625e72fe420597d1b3d17f8c10b2...600507231cb64d38abc713d0e5cbc6c3/)
   [Failed :arrow_down:0.0% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/7e578ec1a56a4051a663ff5957394eb6...2f997c8652374af68f5953573e9a6528/)
   [Finished :arrow_down:0.38% :arrow_up:0.0%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/4a95dba22d49415a8f87732398489617...626b60656d1a42f4a18f25bfe93d01e7/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] ursabot commented on pull request #12603: ARROW-15573: [Java][Doc] Document Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#issuecomment-1070939910


   Benchmark runs are scheduled for baseline = 7e70c429ccd67c35651a8891cff5f7983978a70b and contender = 8db93a5328f93255f352d97b7a5a7b01a1368c6e. 8db93a5328f93255f352d97b7a5a7b01a1368c6e is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Scheduled] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/d40b6b17b1f942229069705150ad6ec4...a66512ee5f954d5d92977e9f0d7e4572/)
   [Scheduled] [test-mac-arm](https://conbench.ursa.dev/compare/runs/670a625e72fe420597d1b3d17f8c10b2...600507231cb64d38abc713d0e5cbc6c3/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/7e578ec1a56a4051a663ff5957394eb6...2f997c8652374af68f5953573e9a6528/)
   [Scheduled] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/4a95dba22d49415a8f87732398489617...626b60656d1a42f4a18f25bfe93d01e7/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] ursabot edited a comment on pull request #12603: ARROW-15573: [Java][Doc] Document Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#issuecomment-1070939910






-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r827444767



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+As an example of child allocator consider `Flight Client`_ creation.

Review comment:
       Deleted




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r827442810



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator

Review comment:
       Updated

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf

Review comment:
       Updated

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+

Review comment:
       Added




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828481796



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,208 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block::
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow modules define simple logging facade for java SLF4J, configure it properly to see your logs (e.g. Logback/Log4J).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    import org.apache.arrow.vector.IntVector;
+
+    try (BufferAllocator bufferAllocator = new RootAllocator(Integer.MAX_VALUE)) {
+        final int QUANTITY = 5;
+        try (IntVector intVector = new IntVector("int-01", bufferAllocator)) {
+            intVector.allocateNew(QUANTITY);
+            for (int i = 0; i < QUANTITY; i++) {
+                intVector.set(i, i);
+            }
+            intVector.setValueCount(QUANTITY);
+        }
+        // Fix the next code!, it is only to see the track of allocators when debug is enabled
+        IntVector intVectorV = new IntVector("int-02", bufferAllocator);
+        intVectorV.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV.set(i, i);
+        }
+        intVectorV.setValueCount(QUANTITY);
+
+        BufferAllocator childAllocator = bufferAllocator.newChildAllocator("child-isolated", 0,
+                Integer.MAX_VALUE / 4);
+        IntVector intVectorV2 = new IntVector("int-isolated-01", childAllocator);
+        intVectorV2.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV2.set(i, i);
+        }
+    }

Review comment:
       Thanks.
   
   I'm pretty sure you can get a stack trace showing where each buffer was allocated, but that may only be in the debugger. I'll try to follow up tomorrow…




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r829087632



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,189 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block:: shell
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with `ReferenceManager.release`_ and
+`ReferenceManager.retain`_.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow logs some allocation information via SLF4J; configure it properly to see these logs (e.g. via Logback/Apache Log4j).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+    }
+
+Logs without debug allocators enabled:
+
+.. code-block:: shell
+
+    11:56:48.944 [main] INFO  o.apache.arrow.memory.BaseAllocator - Debug mode disabled.
+    ArrowBuf[2], address:140508391276544, length:4096
+    16:28:08.847 [main] ERROR o.apache.arrow.memory.BaseAllocator - Memory was leaked by query. Memory leaked: (4096)
+    Allocator(ROOT) 0/4096/4096/8192 (res/actual/peak/limit)
+
+Logs with debug allocators enabled:
+
+.. code-block:: shell
+
+    11:56:48.944 [main] INFO  o.apache.arrow.memory.BaseAllocator - Debug mode enabled.
+    ArrowBuf[2], address:140437894463488, length:4096
+    Exception in thread "main" java.lang.IllegalStateException: Allocator[ROOT] closed with outstanding buffers allocated (1).
+    Allocator(ROOT) 0/4096/4096/8192 (res/actual/peak/limit)
+      child allocators: 0
+      ledgers: 1
+        ledger[1] allocator: ROOT), isOwning: , size: , references: 1, life: 261438177096661..0, allocatorManager: [, life: ] holds 1 buffers.
+            ArrowBuf[2], address:140437894463488, length:4096
+      reservations: 0
+
+.. _`BufferAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/BufferAllocator.html
+.. _`RootAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/RootAllocator.html
+.. _`newChildAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/RootAllocator.html#newChildAllocator-java.lang.String-org.apache.arrow.memory.AllocationListener-long-long-
+.. _`ArrowBuf`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/ArrowBuf.html
+.. _`Reference Counting`: https://github.com/apache/arrow/blob/2092e18752a9c0494799493b12eb1830052217a2/java/memory/memory-core/src/main/java/org/apache/arrow/memory/ReferenceManager.java#L30

Review comment:
       Deleted




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828469588



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,208 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block::
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow modules define simple logging facade for java SLF4J, configure it properly to see your logs (e.g. Logback/Log4J).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    import org.apache.arrow.vector.IntVector;
+
+    try (BufferAllocator bufferAllocator = new RootAllocator(Integer.MAX_VALUE)) {
+        final int QUANTITY = 5;
+        try (IntVector intVector = new IntVector("int-01", bufferAllocator)) {
+            intVector.allocateNew(QUANTITY);
+            for (int i = 0; i < QUANTITY; i++) {
+                intVector.set(i, i);
+            }
+            intVector.setValueCount(QUANTITY);
+        }
+        // Fix the next code!, it is only to see the track of allocators when debug is enabled
+        IntVector intVectorV = new IntVector("int-02", bufferAllocator);
+        intVectorV.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV.set(i, i);
+        }
+        intVectorV.setValueCount(QUANTITY);
+
+        BufferAllocator childAllocator = bufferAllocator.newChildAllocator("child-isolated", 0,
+                Integer.MAX_VALUE / 4);
+        IntVector intVectorV2 = new IntVector("int-isolated-01", childAllocator);
+        intVectorV2.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV2.set(i, i);
+        }
+    }

Review comment:
       Added a example with/without use this parameter




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r829062849



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,208 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block::
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow modules define simple logging facade for java SLF4J, configure it properly to see your logs (e.g. Logback/Log4J).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    import org.apache.arrow.vector.IntVector;
+
+    try (BufferAllocator bufferAllocator = new RootAllocator(Integer.MAX_VALUE)) {
+        final int QUANTITY = 5;
+        try (IntVector intVector = new IntVector("int-01", bufferAllocator)) {
+            intVector.allocateNew(QUANTITY);
+            for (int i = 0; i < QUANTITY; i++) {
+                intVector.set(i, i);
+            }
+            intVector.setValueCount(QUANTITY);
+        }
+        // Fix the next code!, it is only to see the track of allocators when debug is enabled
+        IntVector intVectorV = new IntVector("int-02", bufferAllocator);
+        intVectorV.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV.set(i, i);
+        }
+        intVectorV.setValueCount(QUANTITY);
+
+        BufferAllocator childAllocator = bufferAllocator.newChildAllocator("child-isolated", 0,
+                Integer.MAX_VALUE / 4);
+        IntVector intVectorV2 = new IntVector("int-isolated-01", childAllocator);
+        intVectorV2.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV2.set(i, i);
+        }
+    }

Review comment:
       I'll submit a jira/PR for this later (there's some issues that need fixing) but I think things are good here as is. 




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm closed pull request #12603: ARROW-15573: [Java][Doc] Document Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm closed pull request #12603:
URL: https://github.com/apache/arrow/pull/12603


   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] ursabot edited a comment on pull request #12603: ARROW-15573: [Java][Doc] Document Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#issuecomment-1070939910


   Benchmark runs are scheduled for baseline = 7e70c429ccd67c35651a8891cff5f7983978a70b and contender = 8db93a5328f93255f352d97b7a5a7b01a1368c6e. 8db93a5328f93255f352d97b7a5a7b01a1368c6e is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/d40b6b17b1f942229069705150ad6ec4...a66512ee5f954d5d92977e9f0d7e4572/)
   [Finished :arrow_down:0.29% :arrow_up:0.08%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/670a625e72fe420597d1b3d17f8c10b2...600507231cb64d38abc713d0e5cbc6c3/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/7e578ec1a56a4051a663ff5957394eb6...2f997c8652374af68f5953573e9a6528/)
   [Finished :arrow_down:0.38% :arrow_up:0.0%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/4a95dba22d49415a8f87732398489617...626b60656d1a42f4a18f25bfe93d01e7/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm commented on pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#issuecomment-1064531314


   Sorry, just a note: please try to ping people in a separate comment since the PR description will get put in the eventual commit message and that leads to notification spam (and we don't always remember to remove the ping before merging)


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r827443835



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).

Review comment:
       Deleted and change by suggested

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+

Review comment:
       Thank you

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.

Review comment:
       Added

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+As an example of child allocator consider `Flight Client`_ creation.
+
+Memory Modules
+==============
+
+Applications should depend on memory-core and one of the two implementations,
+else an exception will be raised at runtime.

Review comment:
       Deleted




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r827444486



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+As an example of child allocator consider `Flight Client`_ creation.
+
+Memory Modules
+==============
+
+Applications should depend on memory-core and one of the two implementations,
+else an exception will be raised at runtime.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one allocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+* Allocators will check for outstanding memory allocations when closed, and throw an exception if there are allocated buffers, this helps detect memory leaks.
+* Allocators have a debug mode, that makes it easier to figure out where a leak originated (Consider to add this parameter to your application: -Darrow.memory.debug.allocator=true)

Review comment:
       Added Debugging Memory Leaks/Allocation

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+As an example of child allocator consider `Flight Client`_ creation.
+
+Memory Modules
+==============
+
+Applications should depend on memory-core and one of the two implementations,
+else an exception will be raised at runtime.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one allocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+* Allocators will check for outstanding memory allocations when closed, and throw an exception if there are allocated buffers, this helps detect memory leaks.
+* Allocators have a debug mode, that makes it easier to figure out where a leak originated (Consider to add this parameter to your application: -Darrow.memory.debug.allocator=true)
+* Arrow modules use logback to collect logs configure that properly to see your logs (create logback-test.xml file on resources folder).

Review comment:
       Added Debugging Memory Leaks/Allocation




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r829062849



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,208 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block::
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow modules define simple logging facade for java SLF4J, configure it properly to see your logs (e.g. Logback/Log4J).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    import org.apache.arrow.vector.IntVector;
+
+    try (BufferAllocator bufferAllocator = new RootAllocator(Integer.MAX_VALUE)) {
+        final int QUANTITY = 5;
+        try (IntVector intVector = new IntVector("int-01", bufferAllocator)) {
+            intVector.allocateNew(QUANTITY);
+            for (int i = 0; i < QUANTITY; i++) {
+                intVector.set(i, i);
+            }
+            intVector.setValueCount(QUANTITY);
+        }
+        // Fix the next code!, it is only to see the track of allocators when debug is enabled
+        IntVector intVectorV = new IntVector("int-02", bufferAllocator);
+        intVectorV.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV.set(i, i);
+        }
+        intVectorV.setValueCount(QUANTITY);
+
+        BufferAllocator childAllocator = bufferAllocator.newChildAllocator("child-isolated", 0,
+                Integer.MAX_VALUE / 4);
+        IntVector intVectorV2 = new IntVector("int-isolated-01", childAllocator);
+        intVectorV2.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV2.set(i, i);
+        }
+    }

Review comment:
       I'll submit a jira/PR for this later (there's some issues that need fixing) but I think things are good here as is. 

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,208 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block::
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow modules define simple logging facade for java SLF4J, configure it properly to see your logs (e.g. Logback/Log4J).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    import org.apache.arrow.vector.IntVector;
+
+    try (BufferAllocator bufferAllocator = new RootAllocator(Integer.MAX_VALUE)) {
+        final int QUANTITY = 5;
+        try (IntVector intVector = new IntVector("int-01", bufferAllocator)) {
+            intVector.allocateNew(QUANTITY);
+            for (int i = 0; i < QUANTITY; i++) {
+                intVector.set(i, i);
+            }
+            intVector.setValueCount(QUANTITY);
+        }
+        // Fix the next code!, it is only to see the track of allocators when debug is enabled
+        IntVector intVectorV = new IntVector("int-02", bufferAllocator);
+        intVectorV.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV.set(i, i);
+        }
+        intVectorV.setValueCount(QUANTITY);
+
+        BufferAllocator childAllocator = bufferAllocator.newChildAllocator("child-isolated", 0,
+                Integer.MAX_VALUE / 4);
+        IntVector intVectorV2 = new IntVector("int-isolated-01", childAllocator);
+        intVectorV2.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV2.set(i, i);
+        }
+    }

Review comment:
       I'll submit a jira/PR for this later (there's some issues that need fixing for the historical log) but I think things are good here as is. 

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,189 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block:: shell
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with `ReferenceManager.release`_ and
+`ReferenceManager.retain`_.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow logs some allocation information via SLF4J; configure it properly to see these logs (e.g. via Logback/Apache Log4j).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+    }
+
+Logs without debug allocators enabled:
+
+.. code-block:: shell
+
+    11:56:48.944 [main] INFO  o.apache.arrow.memory.BaseAllocator - Debug mode disabled.
+    ArrowBuf[2], address:140508391276544, length:4096
+    16:28:08.847 [main] ERROR o.apache.arrow.memory.BaseAllocator - Memory was leaked by query. Memory leaked: (4096)
+    Allocator(ROOT) 0/4096/4096/8192 (res/actual/peak/limit)
+
+Logs with debug allocators enabled:
+
+.. code-block:: shell
+
+    11:56:48.944 [main] INFO  o.apache.arrow.memory.BaseAllocator - Debug mode enabled.
+    ArrowBuf[2], address:140437894463488, length:4096
+    Exception in thread "main" java.lang.IllegalStateException: Allocator[ROOT] closed with outstanding buffers allocated (1).
+    Allocator(ROOT) 0/4096/4096/8192 (res/actual/peak/limit)
+      child allocators: 0
+      ledgers: 1
+        ledger[1] allocator: ROOT), isOwning: , size: , references: 1, life: 261438177096661..0, allocatorManager: [, life: ] holds 1 buffers.
+            ArrowBuf[2], address:140437894463488, length:4096
+      reservations: 0
+
+.. _`BufferAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/BufferAllocator.html
+.. _`RootAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/RootAllocator.html
+.. _`newChildAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/RootAllocator.html#newChildAllocator-java.lang.String-org.apache.arrow.memory.AllocationListener-long-long-
+.. _`ArrowBuf`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/ArrowBuf.html
+.. _`Reference Counting`: https://github.com/apache/arrow/blob/2092e18752a9c0494799493b12eb1830052217a2/java/memory/memory-core/src/main/java/org/apache/arrow/memory/ReferenceManager.java#L30

Review comment:
       Wait - I don't think we should link into source files. They don't really provide useful context in this tutorial. (That goes for all the links here.)




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] ursabot edited a comment on pull request #12603: ARROW-15573: [Java][Doc] Document Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#issuecomment-1070939910


   Benchmark runs are scheduled for baseline = 7e70c429ccd67c35651a8891cff5f7983978a70b and contender = 8db93a5328f93255f352d97b7a5a7b01a1368c6e. 8db93a5328f93255f352d97b7a5a7b01a1368c6e is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/d40b6b17b1f942229069705150ad6ec4...a66512ee5f954d5d92977e9f0d7e4572/)
   [Scheduled] [test-mac-arm](https://conbench.ursa.dev/compare/runs/670a625e72fe420597d1b3d17f8c10b2...600507231cb64d38abc713d0e5cbc6c3/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/7e578ec1a56a4051a663ff5957394eb6...2f997c8652374af68f5953573e9a6528/)
   [Scheduled] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/4a95dba22d49415a8f87732398489617...626b60656d1a42f4a18f25bfe93d01e7/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] ursabot commented on pull request #12603: ARROW-15573: [Java][Doc] Document Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#issuecomment-1070939910


   Benchmark runs are scheduled for baseline = 7e70c429ccd67c35651a8891cff5f7983978a70b and contender = 8db93a5328f93255f352d97b7a5a7b01a1368c6e. 8db93a5328f93255f352d97b7a5a7b01a1368c6e is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Scheduled] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/d40b6b17b1f942229069705150ad6ec4...a66512ee5f954d5d92977e9f0d7e4572/)
   [Scheduled] [test-mac-arm](https://conbench.ursa.dev/compare/runs/670a625e72fe420597d1b3d17f8c10b2...600507231cb64d38abc713d0e5cbc6c3/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/7e578ec1a56a4051a663ff5957394eb6...2f997c8652374af68f5953573e9a6528/)
   [Scheduled] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/4a95dba22d49415a8f87732398489617...626b60656d1a42f4a18f25bfe93d01e7/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] ursabot edited a comment on pull request #12603: ARROW-15573: [Java][Doc] Document Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#issuecomment-1070939910


   Benchmark runs are scheduled for baseline = 7e70c429ccd67c35651a8891cff5f7983978a70b and contender = 8db93a5328f93255f352d97b7a5a7b01a1368c6e. 8db93a5328f93255f352d97b7a5a7b01a1368c6e is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/d40b6b17b1f942229069705150ad6ec4...a66512ee5f954d5d92977e9f0d7e4572/)
   [Scheduled] [test-mac-arm](https://conbench.ursa.dev/compare/runs/670a625e72fe420597d1b3d17f8c10b2...600507231cb64d38abc713d0e5cbc6c3/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/7e578ec1a56a4051a663ff5957394eb6...2f997c8652374af68f5953573e9a6528/)
   [Finished :arrow_down:0.38% :arrow_up:0.0%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/4a95dba22d49415a8f87732398489617...626b60656d1a42f4a18f25bfe93d01e7/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r826914806



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_

Review comment:
       Let's not link to Netty's docs right off the bat, I think that's a little confusing without context

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+

Review comment:
       More context here as well. "Direct memory is more expensive to allocate and deallocate, so allocators pool/cache allocated buffers." (We could move Allocator before this section.) "Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to the buffer, and code is responsible for properly incrementing/decrementing the counter as the buffer is used. (Some programming languages offer this as a general memory management strategy, albeit usually hidden from the user to some extent.) 
   
   In Arrow, each buffer has an associated ReferenceManager that tracks the reference count, which can be retrieved with ArrowBuf.getReferenceManager(). The reference count can be updated with ReferenceManager.release and ReferenceManager.retain. Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we use higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically decrement the reference count when closed."
   
   also feel free to rewrite things, I'm just trying to sketch out the amount of context I think we should provide

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:

Review comment:
       "Arrow provides multiple modules: the core interfaces, and implementations of the interfaces. Users need the core interfaces, and exactly one of the implementations."

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+As an example of child allocator consider `Flight Client`_ creation.
+
+Memory Modules
+==============
+
+Applications should depend on memory-core and one of the two implementations,
+else an exception will be raised at runtime.

Review comment:
       This is redundant with above.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+As an example of child allocator consider `Flight Client`_ creation.
+
+Memory Modules
+==============
+
+Applications should depend on memory-core and one of the two implementations,
+else an exception will be raised at runtime.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one allocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+* Allocators will check for outstanding memory allocations when closed, and throw an exception if there are allocated buffers, this helps detect memory leaks.
+* Allocators have a debug mode, that makes it easier to figure out where a leak originated (Consider to add this parameter to your application: -Darrow.memory.debug.allocator=true)
+* Arrow modules use logback to collect logs configure that properly to see your logs (create logback-test.xml file on resources folder).

Review comment:
       I think this can also go as part of a section about debugging memory leaks/allocations

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+As an example of child allocator consider `Flight Client`_ creation.
+
+Memory Modules
+==============
+
+Applications should depend on memory-core and one of the two implementations,
+else an exception will be raised at runtime.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one allocator at the start of the program.

Review comment:
       ```suggestion
   * Applications should generally create one RootAllocator at the start of the program.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.

Review comment:
       ```suggestion
   Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,

Review comment:
       nit: link to the API docs again here

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).

Review comment:
       Is this the case? As described this sounds like a reference cycle.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.

Review comment:
       "Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers obtained from the allocator are closed. If not, close() will raise an exception; this helps track memory leaks from unclosed buffers. When combined with child allocators, this is especially useful for ensuring at an independent section of code has fully cleaned up all allocated buffers, while still maintaining a global memory limit through the RootAllocator."

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+As an example of child allocator consider `Flight Client`_ creation.
+
+Memory Modules
+==============
+
+Applications should depend on memory-core and one of the two implementations,
+else an exception will be raised at runtime.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one allocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+* Allocators will check for outstanding memory allocations when closed, and throw an exception if there are allocated buffers, this helps detect memory leaks.
+* Allocators have a debug mode, that makes it easier to figure out where a leak originated (Consider to add this parameter to your application: -Darrow.memory.debug.allocator=true)

Review comment:
       What does the debug mode do? Also, I think this can be a subsection of the Allocators section, since we should expand on it

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,

Review comment:
       We need more context here in general. We should link "direct memory" to https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/nio/ByteBuffer.html since it explains direct byte buffers.
   
   We should also briefly summarize what direct memory is, and why Arrow uses it.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.

Review comment:
       "The BufferAllocator interface deals with allocating ArrowBufs for the application."

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+As an example of child allocator consider `Flight Client`_ creation.

Review comment:
       IMO, pointing into internal implementation details does not make for a good example. We should add and link to a cookbook example.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..

Review comment:
       ```suggestion
   The concrete implementation of the allocator is RootAllocator. Applications should generally
   create one allocator at the start of the program, and use it through the BufferAllocator interface.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible

Review comment:
       Add the method used to create child allocators. 
   
   Mention that allocators have a memory limit. The RootAllocator then effectively sets the program-wide memory limit. One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+As an example of child allocator consider `Flight Client`_ creation.
+
+Memory Modules
+==============
+
+Applications should depend on memory-core and one of the two implementations,
+else an exception will be raised at runtime.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one allocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+* Allocators will check for outstanding memory allocations when closed, and throw an exception if there are allocated buffers, this helps detect memory leaks.

Review comment:
       This isn't a guideline IMO (it should be something specific and concrete that developers should do, not a description of the Arrow implementation)




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r827443428



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:

Review comment:
       Updated

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,

Review comment:
       Added




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r827954312



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.

Review comment:
       Use the actual module name, `arrow-memory-core` etc.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.

Review comment:
       ```suggestion
   Arrow offers a high level of abstraction providing several access APIs to read/write data into direct memory.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).
+* Conversely in JNI, we can directly use `Java ArrowBufs in C++`_ without having to copy data.
+
+So basically #1 is more efficient I/O, and #2/#3 is better integration with JNI code.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one RootAllocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.

Review comment:
       ```suggestion
   Applications should generally:
   * Use the BufferAllocator interface in APIs instead of RootAllocator.
   * Create one RootAllocator at the start of the program.
   * ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================

Review comment:
       ```suggestion
   Why Arrow Uses Direct Memory
   ============================
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+

Review comment:
       Add a paragraph/short snippet demonstrating allocating a buffer from an allocator.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_

Review comment:
       Let's not link into random source files either. We don't have to link everything here.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.

Review comment:
       ```suggestion
   decrement the reference count when closed.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,

Review comment:
       ```suggestion
   ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.

Review comment:
       On second thought, maybe just remove this paragraph.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).
+* Conversely in JNI, we can directly use `Java ArrowBufs in C++`_ without having to copy data.
+
+So basically #1 is more efficient I/O, and #2/#3 is better integration with JNI code.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one RootAllocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode, that makes it easier to figure out where a leak is originated (Consider to add this
+parameter to your application: ``-Darrow.memory.debug.allocator=true``). This parameter enable to create an historical log
+about the memory allocation.
+
+Arrow modules use logback to collect logs, configure it properly to see your logs (create ``logback-test.xml`` file on
+resources folder and your project could read that by conventions).

Review comment:
       We use slf4j. Applications can configure their favorite backend.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).
+* Conversely in JNI, we can directly use `Java ArrowBufs in C++`_ without having to copy data.
+
+So basically #1 is more efficient I/O, and #2/#3 is better integration with JNI code.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one RootAllocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode, that makes it easier to figure out where a leak is originated (Consider to add this
+parameter to your application: ``-Darrow.memory.debug.allocator=true``). This parameter enable to create an historical log
+about the memory allocation.
+
+Arrow modules use logback to collect logs, configure it properly to see your logs (create ``logback-test.xml`` file on
+resources folder and your project could read that by conventions).
+
+This is an example of historical log enabled:

Review comment:
       I think this is the regular exception you get, this doesn't show any extra debug info.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).

Review comment:
       ```suggestion
   Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).
+* Conversely in JNI, we can directly use `Java ArrowBufs in C++`_ without having to copy data.
+
+So basically #1 is more efficient I/O, and #2/#3 is better integration with JNI code.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one RootAllocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode, that makes it easier to figure out where a leak is originated (Consider to add this
+parameter to your application: ``-Darrow.memory.debug.allocator=true``). This parameter enable to create an historical log
+about the memory allocation.

Review comment:
       ```suggestion
   Allocators have a debug mode that makes it easier to figure out where a leak is originated.
   To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
   When enabled, a log will be kept of allocations.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).

Review comment:
       I think we can remove this, and move the section on direct memory below to be a subsection at the end of this section.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf

Review comment:
       I think this should go first

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).
+* Conversely in JNI, we can directly use `Java ArrowBufs in C++`_ without having to copy data.

Review comment:
       "Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data."

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).

Review comment:
       "Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface."

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.

Review comment:
       Link to BufferAllocator's javadocs.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.

Review comment:
       We don't need to link into random bits of code in the user-facing docs.
   
   "The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow." 

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.

Review comment:
       Hmm, I feel like we can still introduce this better.
   
   "The concrete implementation of the BufferAllocator interface is RootAllocator. (Link to the Javadocs.) Applications should generally create one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory has been freed (see the next section).
   
   Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
   are created as children of an existing allocator via newChildAllocator. (Link to Javadocs.) When creating a RootAllocator or a child allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
   
   Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging."

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).

Review comment:
       Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers. The caching is not the _reason_ why it's expensive, it's the _consequence_.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned

Review comment:
       Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of the garbage collector.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.

Review comment:
       ```suggestion
   the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and

Review comment:
       Link to Javadocs.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )

Review comment:
       This is wrong, no? Buffers have a reference count, not vectors. And ValidityBuffer and ValueBuffer aren't classes.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.

Review comment:
       ```suggestion
   As you see, reference counting needs to be handled carefully. To ensure that an
   independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).
+* Conversely in JNI, we can directly use `Java ArrowBufs in C++`_ without having to copy data.
+
+So basically #1 is more efficient I/O, and #2/#3 is better integration with JNI code.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one RootAllocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode, that makes it easier to figure out where a leak is originated (Consider to add this
+parameter to your application: ``-Darrow.memory.debug.allocator=true``). This parameter enable to create an historical log
+about the memory allocation.

Review comment:
       Have you tried this? Can you show exactly 1) how to access this log and 2) what it looks like?

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_

Review comment:
       Or actually, if you want: we can remove this list and put `.. contents::` here instead.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).
+* Conversely in JNI, we can directly use `Java ArrowBufs in C++`_ without having to copy data.
+
+So basically #1 is more efficient I/O, and #2/#3 is better integration with JNI code.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one RootAllocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode, that makes it easier to figure out where a leak is originated (Consider to add this
+parameter to your application: ``-Darrow.memory.debug.allocator=true``). This parameter enable to create an historical log
+about the memory allocation.
+
+Arrow modules use logback to collect logs, configure it properly to see your logs (create ``logback-test.xml`` file on
+resources folder and your project could read that by conventions).
+
+This is an example of historical log enabled:

Review comment:
       Also we should provide the corresponding code snippet.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,

Review comment:
       I think upon digging around that multiple ArrowBufs can point to the same allocation (that is why the reference counting is handled by a separate ReferenceManager)




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r826443988



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.
+
+.. code-block:: Java
+
+    try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE) ) { ; }
+
+ArrowBuf
+========
+
+The facade for interacting directly with a chunk of memory.
+
+Two important instance variables of an ArrowBuf:
+
+* Address: Starting virtual address in the underlying memory chunk that this ArrowBuf has access to.
+* Length: Length (in bytes) in the underlying memory chunk that this ArrowBuf has access to.
+

Review comment:
       Added

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:

Review comment:
       Updated




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r827444201



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,114 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `Reference counting`_
+* `BufferAllocator`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offer a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules, but users only need two of them:
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section).
+
+Reference counting
+==================
+
+Is a technique to help computer programs manage memory. Tracks the reference/pointers to an object, it increase
++1 or decrease -1 the reference counting between the objects.
+
+If an object ValidityBuffer has a reference with object IntVector, then, IntVector should increase the
+reference counting to 1 (0 + 1 = 1), then if at the same time, ValueBuffer has a reference with IntVector,
+then, IntVector should increase the reference counting to 2 (1 + 1 = 2).
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Base on best practices at some point you are going to close your allocator objects using ``close()`` method,
+allocators check for reference counting and throw an exception if they are in use.
+
+Reference Manager manages the reference counting for the underlying memory chunk.
+
+Allocators
+==========
+
+One of the interfaces defined by memory-core is BufferAllocator. This interface collect all the definitions for deal
+with byte buffer allocation.
+
+The concrete implementation of the allocator is Root Allocator. Applications should generally
+create one allocator at the start of the program..
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+As an example of child allocator consider `Flight Client`_ creation.
+
+Memory Modules
+==============
+
+Applications should depend on memory-core and one of the two implementations,
+else an exception will be raised at runtime.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one allocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+* Allocators will check for outstanding memory allocations when closed, and throw an exception if there are allocated buffers, this helps detect memory leaks.

Review comment:
       Deleted




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828341138



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf

Review comment:
       Changed




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm closed pull request #12603: ARROW-15573: [Java][Doc] Document Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm closed pull request #12603:
URL: https://github.com/apache/arrow/pull/12603


   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r829087632



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,189 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block:: shell
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with `ReferenceManager.release`_ and
+`ReferenceManager.retain`_.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow logs some allocation information via SLF4J; configure it properly to see these logs (e.g. via Logback/Apache Log4j).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+    }
+
+Logs without debug allocators enabled:
+
+.. code-block:: shell
+
+    11:56:48.944 [main] INFO  o.apache.arrow.memory.BaseAllocator - Debug mode disabled.
+    ArrowBuf[2], address:140508391276544, length:4096
+    16:28:08.847 [main] ERROR o.apache.arrow.memory.BaseAllocator - Memory was leaked by query. Memory leaked: (4096)
+    Allocator(ROOT) 0/4096/4096/8192 (res/actual/peak/limit)
+
+Logs with debug allocators enabled:
+
+.. code-block:: shell
+
+    11:56:48.944 [main] INFO  o.apache.arrow.memory.BaseAllocator - Debug mode enabled.
+    ArrowBuf[2], address:140437894463488, length:4096
+    Exception in thread "main" java.lang.IllegalStateException: Allocator[ROOT] closed with outstanding buffers allocated (1).
+    Allocator(ROOT) 0/4096/4096/8192 (res/actual/peak/limit)
+      child allocators: 0
+      ledgers: 1
+        ledger[1] allocator: ROOT), isOwning: , size: , references: 1, life: 261438177096661..0, allocatorManager: [, life: ] holds 1 buffers.
+            ArrowBuf[2], address:140437894463488, length:4096
+      reservations: 0
+
+.. _`BufferAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/BufferAllocator.html
+.. _`RootAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/RootAllocator.html
+.. _`newChildAllocator`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/RootAllocator.html#newChildAllocator-java.lang.String-org.apache.arrow.memory.AllocationListener-long-long-
+.. _`ArrowBuf`: https://arrow.apache.org/docs/java/reference/org/apache/arrow/memory/ArrowBuf.html
+.. _`Reference Counting`: https://github.com/apache/arrow/blob/2092e18752a9c0494799493b12eb1830052217a2/java/memory/memory-core/src/main/java/org/apache/arrow/memory/ReferenceManager.java#L30

Review comment:
       Deleted




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828340083



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.

Review comment:
       Deleted

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.

Review comment:
       Added

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned

Review comment:
       Added

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).

Review comment:
       Updated

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.

Review comment:
       Thanks, updated

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and

Review comment:
       Added

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+

Review comment:
       Added snippet code

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).
+* Conversely in JNI, we can directly use `Java ArrowBufs in C++`_ without having to copy data.
+
+So basically #1 is more efficient I/O, and #2/#3 is better integration with JNI code.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one RootAllocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode, that makes it easier to figure out where a leak is originated (Consider to add this
+parameter to your application: ``-Darrow.memory.debug.allocator=true``). This parameter enable to create an historical log
+about the memory allocation.
+
+Arrow modules use logback to collect logs, configure it properly to see your logs (create ``logback-test.xml`` file on
+resources folder and your project could read that by conventions).

Review comment:
       Updated

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).

Review comment:
       Thank you, added on my notes. Updated




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828341417



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.

Review comment:
       Thanks, added

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).
+* Conversely in JNI, we can directly use `Java ArrowBufs in C++`_ without having to copy data.

Review comment:
       Updated




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828341058



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed method.
+
+.. code-block::
+
+    |__ A = Allocator
+    |____ B = IntVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+    |____ C = VarcharVector (reference count = 2 )
+    |____________ ValidityBuffer
+    |____________ ValueBuffer
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see reference counting needs to be handled properly by us, if at some point you need to ensuring that an
+independent section of code has `fully cleaned up all allocated buffers while still maintaining a global memory limit
+through the RootAllocator`, well ``BufferAllocator.newChildAllocator`` is what you should use.
+
+Reason To Use Direct Memory
+===========================
+
+* When `writing an ArrowBuf`_ we use the direct buffer (``nioBuffer()`` returns a DirectByteBuffer) and the JVM `will attempt to avoid copying the buffer's content to (or from) an intermediate buffer`_ so it makes I/O (and hence IPC) faster.
+* We can `directly wrap a native memory address`_ instead of having to copy data for JNI (where in implementing the C Data Interface we can directly create `Java ArrowBufs that directly correspond to the C pointers`_).
+* Conversely in JNI, we can directly use `Java ArrowBufs in C++`_ without having to copy data.
+
+So basically #1 is more efficient I/O, and #2/#3 is better integration with JNI code.
+
+Development Guidelines
+======================
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Applications should generally create one RootAllocator at the start of the program.
+* Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode, that makes it easier to figure out where a leak is originated (Consider to add this
+parameter to your application: ``-Darrow.memory.debug.allocator=true``). This parameter enable to create an historical log
+about the memory allocation.
+
+Arrow modules use logback to collect logs, configure it properly to see your logs (create ``logback-test.xml`` file on
+resources folder and your project could read that by conventions).
+
+This is an example of historical log enabled:

Review comment:
       Added snippet code




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r828399202



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,208 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block::
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow modules define simple logging facade for java SLF4J, configure it properly to see your logs (e.g. Logback/Log4J).

Review comment:
       We don't define slf4j.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,208 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block::
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow modules define simple logging facade for java SLF4J, configure it properly to see your logs (e.g. Logback/Log4J).

Review comment:
       ```suggestion
   Arrow logs some allocation information via SLF4J; configure it properly to see these logs (e.g. via Logback/Apache Log4j).
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,208 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block::
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow modules define simple logging facade for java SLF4J, configure it properly to see your logs (e.g. Logback/Log4J).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    import org.apache.arrow.vector.IntVector;
+
+    try (BufferAllocator bufferAllocator = new RootAllocator(Integer.MAX_VALUE)) {
+        final int QUANTITY = 5;
+        try (IntVector intVector = new IntVector("int-01", bufferAllocator)) {
+            intVector.allocateNew(QUANTITY);
+            for (int i = 0; i < QUANTITY; i++) {
+                intVector.set(i, i);
+            }
+            intVector.setValueCount(QUANTITY);
+        }
+        // Fix the next code!, it is only to see the track of allocators when debug is enabled
+        IntVector intVectorV = new IntVector("int-02", bufferAllocator);
+        intVectorV.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV.set(i, i);
+        }
+        intVectorV.setValueCount(QUANTITY);
+
+        BufferAllocator childAllocator = bufferAllocator.newChildAllocator("child-isolated", 0,
+                Integer.MAX_VALUE / 4);
+        IntVector intVectorV2 = new IntVector("int-isolated-01", childAllocator);
+        intVectorV2.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV2.set(i, i);
+        }
+    }

Review comment:
       We should only use ArrowBuf here, don't use IntVector or anything as they haven't been introduced yet.
   
   Also, it's still not clear to me what the debug mode is doing here. From what I recall, the information it provides is basically only available in a debugger. If there is something the debug mode is doing here, we need to point it out explicitly. Ideally we would compare the information available with and without debug mode.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,208 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `ArrowBuf`_
+* `BufferAllocator`_
+* `Reference counting`_
+
+.. contents::
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``memory-core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``memory-netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``memory-unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous region of `direct memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+Unlike (Direct)ByteBuffer, it has reference counting built in, as discussed later.
+
+Why Arrow Uses Direct Memory
+----------------------------
+
+* The JVM can optimize I/O operations when using direct memory/direct buffers; it will attempt to avoid copying buffer contents to/from an intermediate buffer. This can speed up IPC in Arrow.
+* Since Arrow always uses direct memory, JNI modules can directly wrap native memory addresses instead of copying data. We use this in modules like the C Data Interface.
+* Conversely, on the C++ side of the JNI boundary, we can directly access the memory in ArrowBuf without copying data.
+
+BufferAllocator
+===============
+
+The `BufferAllocator`_ interface deals with allocating ArrowBufs for the application.
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.ArrowBuf;
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    try(BufferAllocator bufferAllocator = new RootAllocator(8 * 1024)){
+        ArrowBuf arrowBuf = bufferAllocator.buffer(4 * 1024);
+        System.out.println(arrowBuf);
+        arrowBuf.close();
+    }
+
+.. code-block::
+
+    ArrowBuf[2], address:140363641651200, length:4096
+
+The concrete implementation of the BufferAllocator interface is `RootAllocator`_. Applications should generally create
+one RootAllocator at the start of the program, and use it through the BufferAllocator interface. Allocators implement
+AutoCloseable and must be closed after the application is done with them; this will check that all outstanding memory
+has been freed (see the next section).
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then more allocators
+are created as children of an existing allocator via `newChildAllocator`_. When creating a RootAllocator or a child
+allocator, a memory limit is provided, and when allocating memory, the limit is checked. Furthermore, when allocating
+memory from a child allocator, those allocations are also reflected in all parent allocators. Hence, the RootAllocator
+effectively sets the program-wide memory limit, and serves as the master bookkeeper for all memory allocations.
+
+Child allocators are not strictly required, but can help better organize code. For instance, a lower memory limit can
+be set for a particular section of code. When the allocator is closed, it then checks that that section didn't leak any
+memory. And child allocators can be named, which makes it easier to tell where an ArrowBuf came from during debugging.
+
+Reference counting
+==================
+
+Direct memory is more expensive to allocate and deallocate. That's why allocators pool or cache direct buffers.
+
+Because we want to pool/cache buffers and manage them deterministically, we use manual reference counting instead of
+the garbage collector. This simply means that each buffer has a counter keeping track of the number of references to
+the buffer, and the user is responsible for properly incrementing/decrementing the counter as the buffer is used.
+
+In Arrow, each ArrowBuf has an associated `ReferenceManager`_ that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and
+``ReferenceManager.retain``.
+
+Of course, this is tedious and error-prone, so usually, instead of directly working with buffers, we should use
+higher-level APIs like ValueVector. Such classes generally implement Closeable/AutoCloseable and will automatically
+decrement the reference count when closed.
+
+Allocators implement AutoCloseable as well. In this case, closing the allocator will check that all buffers
+obtained from the allocator are closed. If not, ``close()`` method will raise an exception; this helps track
+memory leaks from unclosed buffers.
+
+As you see, reference counting needs to be handled carefully. To ensure that an
+independent section of code has fully cleaned up all allocated buffers, use a new child allocator.
+
+Development Guidelines
+======================
+
+Applications should generally:
+
+* Use the BufferAllocator interface in APIs instead of RootAllocator.
+* Create one RootAllocator at the start of the program.
+* ``close()`` allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement.
+
+Debugging Memory Leaks/Allocation
+=================================
+
+Allocators have a debug mode that makes it easier to figure out where a leak is originated.
+To enable it, enable assertions with ``-ea`` or set the system property, ``-Darrow.memory.debug.allocator=true``.
+When enabled, a log will be kept of allocations.
+
+Arrow modules define simple logging facade for java SLF4J, configure it properly to see your logs (e.g. Logback/Log4J).
+
+Consider the following example to see how debug enabled help us with the tracking of allocators:
+
+.. code-block:: Java
+
+    import org.apache.arrow.memory.BufferAllocator;
+    import org.apache.arrow.memory.RootAllocator;
+    import org.apache.arrow.vector.IntVector;
+
+    try (BufferAllocator bufferAllocator = new RootAllocator(Integer.MAX_VALUE)) {
+        final int QUANTITY = 5;
+        try (IntVector intVector = new IntVector("int-01", bufferAllocator)) {
+            intVector.allocateNew(QUANTITY);
+            for (int i = 0; i < QUANTITY; i++) {
+                intVector.set(i, i);
+            }
+            intVector.setValueCount(QUANTITY);
+        }
+        // Fix the next code!, it is only to see the track of allocators when debug is enabled
+        IntVector intVectorV = new IntVector("int-02", bufferAllocator);
+        intVectorV.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV.set(i, i);
+        }
+        intVectorV.setValueCount(QUANTITY);
+
+        BufferAllocator childAllocator = bufferAllocator.newChildAllocator("child-isolated", 0,
+                Integer.MAX_VALUE / 4);
+        IntVector intVectorV2 = new IntVector("int-isolated-01", childAllocator);
+        intVectorV2.allocateNew(QUANTITY);
+        for (int i = 0; i < QUANTITY; i++) {
+            intVectorV2.set(i, i);
+        }
+    }
+
+.. code-block::
+
+    15:49:32,755 |-INFO in ch.qos.logback.classic.LoggerContext[default] - Found resource [logback-test.xml] at [file:/Users/java/source/demo/target/classes/logback-test.xml]
+    15:49:32,924 |-INFO in ch.qos.logback.classic.joran.action.LoggerAction - Setting level of logger [org.apache.arrow] to DEBUG
+    11:56:48.944 [main] INFO  o.apache.arrow.memory.BaseAllocator - Debug mode enabled.
+    Exception in thread "main" java.lang.IllegalStateException: Allocator[ROOT] closed with outstanding child allocators.
+    Allocator(ROOT) 0/64/64/2147483647 (res/actual/peak/limit)
+      child allocators: 1
+        Allocator(child-isolated) 0/32/32/536870911 (res/actual/peak/limit)
+          child allocators: 0
+          ledgers: 1
+            ledger[3] allocator: child-isolated), isOwning: , size: , references: 2, life: 246918908438818..0, allocatorManager: [, life: ] holds 3 buffers.
+                ArrowBuf[10], address:140408097079352, length:8
+                ArrowBuf[8], address:140408097079328, length:32
+                ArrowBuf[9], address:140408097079328, length:24
+          reservations: 0
+      ledgers: 1
+        ledger[2] allocator: ROOT), isOwning: , size: , references: 2, life: 246tors can be named; this makes it easier to tell where an Arro918906331643..0, allocatorManager: [, life: ] holds 3 buffers.

Review comment:
       Seems something got copy-pasted into the middle here?

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,174 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+.. contents::
+
+The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
+This section will introduce you to the major concepts in Java’s memory management:
+
+* `BufferAllocator`_
+* `ArrowBuf`_
+* `Reference counting`_
+
+Getting Started
+===============
+
+Arrow's memory management is built around the needs of the columnar format and using off-heap memory.
+Also, it is its own independent implementation, and does not wrap the C++ implementation.
+
+Arrow offers a high level of abstraction providing several access APIs to read/write data into a direct memory.
+
+Arrow provides multiple modules: the core interfaces, and implementations of the interfaces.
+Users need the core interfaces, and exactly one of the implementations.
+
+* ``Memory Core``: Provides the interfaces used by the Arrow libraries and applications.
+* ``Memory Netty``: An implementation of the memory interfaces based on the `Netty`_ library.
+* ``Memory Unsafe``: An implementation of the memory interfaces based on the `sun.misc.Unsafe`_ library.
+
+BufferAllocator
+===============
+
+The BufferAllocator interface deals with allocating ArrowBufs for the application.
+
+The concrete implementation of the allocator is RootAllocator. Applications should generally create one RootAllocator at the
+start of the program, and use it through the BufferAllocator interface. Allocators have a memory limit. The RootAllocator
+sets the program-wide memory limit. The RootAllocator is responsible for being the master bookkeeper for memory allocations.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators
+are created as children ``BufferAllocator.newChildAllocator`` of that allocator.
+
+One of the uses of child allocators is to set a lower temporary limit for one section of the code. Also, child
+allocators can be named; this makes it easier to tell where an ArrowBuf came from during debugging.
+
+ArrowBuf
+========
+
+ArrowBuf represents a single, contiguous allocation of `Direct Memory`_. It consists of an address and a length,
+and provides low-level interfaces for working with the contents, similar to ByteBuffer.
+
+The objects created using ``Direct Memory`` take advantage of native executions and it is decided natively by the JVM. Arrow
+offer efficient memory operations base on this Direct Memory implementation (`see section below for detailed reasons of use`).
+
+Unlike (Direct)ByteBuffer, it has reference counting built in (`see the next section`).
+
+Reference counting
+==================
+
+Direct memory involve more activities than allocate and deallocate because allocators (thru pool/cache)
+allocate buffers (ArrowBuf).
+
+Arrow uses manual reference counting to track whether a buffer is in use, or can be deallocated or returned
+to the allocator's pool. This simply means that each buffer has a counter keeping track of the number of references to
+this buffer, and end user is responsible for properly incrementing/decrementing the counter according the buffer is used.
+
+In Arrow, each ArrowBuf has an associated ReferenceManager that tracks the reference count, which can be retrieved
+with ArrowBuf.getReferenceManager(). The reference count can be updated with ``ReferenceManager.release`` and

Review comment:
       Was this added?




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r824186945



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.

Review comment:
       This sentence is a little redundant

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator

Review comment:
       Maybe `BufferAllocator` and link to the Javadoc

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf

Review comment:
       Maybe `ArrowBuf` and link to the Javadoc

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+

Review comment:
       I would argue that "Reference counting" is a third major concept

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:

Review comment:
       Arrow provides multiple modules, but users only need one or two of them:

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.

Review comment:
       Hmm, the focus here is a little off IMO. Something like this: Arrow's memory management is built around the needs of the columnar format and using off-heap memory. Also, it is its own independent implementation, and does not wrap the C++ implementation.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).

Review comment:
       I'm not really sure what this tells us, what is the user learning here?

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.

Review comment:
       ```
   ``memory-core``: provides the interfaces used by the Arrow libraries and applications.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.

Review comment:
       ```
   ``memory-netty``: an implementation of the memory interfaces based on the [Netty](INSERT LINK HERE) library.
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:

Review comment:
       Again, for a beginner, I think we can have more explanation, and focus less on just replicating the README 1-1

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.

Review comment:
       "The concrete implementation of the allocator. Applications should generally create one allocator at the start of the program." 
   
   The direct memory stuff should be described above, not here.
   
   Also, if we are referencing a class name, then just use the class name (ideally with a link to the docs), e.g. "RootAllocator", else it gets confusing

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.

Review comment:
       and ditto

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.
+
+.. code-block:: Java
+
+    try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE) ) { ; }
+
+ArrowBuf
+========
+
+The facade for interacting directly with a chunk of memory.
+
+Two important instance variables of an ArrowBuf:
+
+* Address: Starting virtual address in the underlying memory chunk that this ArrowBuf has access to.
+* Length: Length (in bytes) in the underlying memory chunk that this ArrowBuf has access to.
+

Review comment:
       As suggested above we should have a section on reference counting. We should talk about remembering to `close()` objects here

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:

Review comment:
       "Consider these guidelines:"

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.

Review comment:
       "Remember to close() allocators after use (whether they are child allocators or the RootAllocator), either manually or preferably via a try-with-resources statement. Allocators will check for outstanding memory allocations when closed, and throw an exception if there are allocated buffers; this helps detect memory leaks." 

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.
+
+.. code-block:: Java
+
+    try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE) ) { ; }
+
+ArrowBuf
+========
+
+The facade for interacting directly with a chunk of memory.
+
+Two important instance variables of an ArrowBuf:
+
+* Address: Starting virtual address in the underlying memory chunk that this ArrowBuf has access to.
+* Length: Length (in bytes) in the underlying memory chunk that this ArrowBuf has access to.

Review comment:
       I don't think we need to elaborate on this so much. 

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.

Review comment:
       "Use the BufferAllocator interface in APIs instead of RootAllocator."

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.
+
+.. code-block:: Java
+
+    try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE) ) { ; }
+
+ArrowBuf
+========
+
+The facade for interacting directly with a chunk of memory.

Review comment:
       "ArrowBuf represents a single, contiguous allocation of direct memory. It consists of an address and a length, and provides low-level interfaces for working with the contents, similar to ByteBuffer. Unlike (Direct)ByteBuffer, it has reference counting built in (see the next section)."

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.

Review comment:
       ```suggestion
   The memory modules contain all the functionality that Arrow uses to manage memory (allocation and deallocation).
   ```

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.
+
+.. code-block:: Java
+
+    try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE) ) { ; }

Review comment:
       Java users know how to use try-with-resources. We should assume they are a beginner to Arrow, but are reasonably familiar with Java itself.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.

Review comment:
       We should elaborate on how and why Arrow uses direct memory

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:

Review comment:
       "One of the interfaces defined by memory-core is BufferAllocator. This class is responsible for..."

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.
+
+.. code-block:: Java
+
+    try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE) ) { ; }
+
+ArrowBuf
+========
+
+The facade for interacting directly with a chunk of memory.
+
+Two important instance variables of an ArrowBuf:
+
+* Address: Starting virtual address in the underlying memory chunk that this ArrowBuf has access to.
+* Length: Length (in bytes) in the underlying memory chunk that this ArrowBuf has access to.
+
+Memory Modules
+==============
+
+Memory core define the bases to work with direct memory and the application decided to allocate arrow buffer bases on
+the dependency added on your pom.xml (memory-unsafe or memory-netty). If any of these is not added the application raise
+an exception.

Review comment:
       This should just be one sentence as part of the list of modules above. "Applications should depend on memory-core and one of the two implementations, else an exception will be raised at runtime."
   
   We generally shouldn't talk about pom.xml. Not all people use Maven.

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.

Review comment:
       Why do we want to use child allocators?

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.
+* Memory Netty: Netty allocator and utils for allocating memory in Arrow.
+* Memory Unsafe: Allocator and utils for allocating memory in Arrow based on sun.misc.Unsafe.
+
+Allocators
+==========
+
+Memory core module define the next allocators:
+
+* Buffer Allocator: The public interface application users should be leveraging.
+* Root Allocator: A root allocator for using direct memory. Typically only one created for a JVM.
+
+Arrow provides a tree-based model for memory allocation. The RootAllocator is created first,
+then all allocators are created as children of that allocator. The RootAllocator is responsible
+for being the master bookkeeper for memory allocations.
+
+Please consider this note on your development:
+
+* Use BufferAllocator instead of RootAllocator in your allocator creation.
+* Create your allocator inside of a try-with-resources statement.

Review comment:
       Allocators have a debug mode, IIRC, that makes it easier to figure out where a leak originated




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] davisusanibar commented on a change in pull request #12603: ARROW-15573: [Java][Doc] Apache Arrow memory management

Posted by GitBox <gi...@apache.org>.
davisusanibar commented on a change in pull request #12603:
URL: https://github.com/apache/arrow/pull/12603#discussion_r826442618



##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).

Review comment:
       Deleted

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:

Review comment:
       Updated

##########
File path: docs/source/java/memory.rst
##########
@@ -0,0 +1,82 @@
+.. Licensed to the Apache Software Foundation (ASF) under one
+.. or more contributor license agreements.  See the NOTICE file
+.. distributed with this work for additional information
+.. regarding copyright ownership.  The ASF licenses this file
+.. to you under the Apache License, Version 2.0 (the
+.. "License"); you may not use this file except in compliance
+.. with the License.  You may obtain a copy of the License at
+
+..   http://www.apache.org/licenses/LICENSE-2.0
+
+.. Unless required by applicable law or agreed to in writing,
+.. software distributed under the License is distributed on an
+.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+.. KIND, either express or implied.  See the License for the
+.. specific language governing permissions and limitations
+.. under the License.
+
+=================
+Memory Management
+=================
+
+The memory management package contains all the memory allocation related items that Arrow uses to manage memory.
+This section will introduce you to the major concepts in Java’s memory management:
+
+* Allocator
+* Arrowbuf
+
+.. contents::
+
+Getting Started
+===============
+
+Java memory implementation is independently from C++ (not a wrapper around).
+Java memory was implemented considering these specifications: Arrow Columnar Format and Java Off Heap references.
+
+.. note::
+
+    Java Memory Data = Data (Columnar mode reference) + Metadata (Flatbuffers serialization reference).
+
+These are the java memory modules:
+
+* Memory Core: Core off-heap memory management libraries for Arrow ValueVectors.

Review comment:
       Updated




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org