You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ra...@apache.org on 2019/07/24 13:50:50 UTC

[arrow] branch master updated: ARROW-5970: [Java] Provide pointer to Arrow buffer

This is an automated email from the ASF dual-hosted git repository.

ravindra pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 065d9dc  ARROW-5970: [Java] Provide pointer to Arrow buffer
065d9dc is described below

commit 065d9dc403dff7f20415e43fb4d63bd7b2d00819
Author: liyafan82 <fa...@foxmail.com>
AuthorDate: Wed Jul 24 19:20:22 2019 +0530

    ARROW-5970: [Java] Provide pointer to Arrow buffer
    
    Introduce pointer to a memory region within an ArrowBuf.
    
    This pointer will be used as the basis for calculating the hash code within a vector, and equality determination.
    
    This data structure can be considered as a "universal value holder".
    
    Closes #4897 from liyafan82/fly_0717_ptr and squashes the following commits:
    
    f9b0ee47f <liyafan82> Merge branch 'master' into fly_0717_ptr
    b2fa206a8 <liyafan82> Merge branch 'master' into fly_0717_ptr
    394b356b6 <liyafan82>  Move ByteFunctionHelpers class to memory module
    7e34ae001 <liyafan82>  Provide pointer to Arrow buffer
    
    Authored-by: liyafan82 <fa...@foxmail.com>
    Signed-off-by: Pindikura Ravindra <ra...@dremio.com>
---
 .../apache/arrow/memory/util/ArrowBufPointer.java  | 106 ++++++++++
 .../arrow/memory}/util/ByteFunctionHelpers.java    |  85 ++++----
 .../arrow/memory/util/TestArrowBufPointer.java     |  70 +++++++
 .../memory}/util/TestByteFunctionHelpers.java      |   2 +-
 .../util/ByteFunctionHelpersBenchmarks.java        |   2 +-
 .../apache/arrow/vector/BaseFixedWidthVector.java  |  18 +-
 .../arrow/vector/BaseVariableWidthVector.java      |  20 +-
 .../org/apache/arrow/vector/FixedWidthVector.java  |  16 ++
 .../apache/arrow/vector/VariableWidthVector.java   |  17 ++
 .../arrow/vector/complex/FixedSizeListVector.java  |   2 +-
 .../arrow/vector/util/ByteFunctionHelpers.java     | 215 ++-------------------
 .../org/apache/arrow/vector/TestValueVector.java   |  70 +++++++
 12 files changed, 377 insertions(+), 246 deletions(-)

diff --git a/java/memory/src/main/java/org/apache/arrow/memory/util/ArrowBufPointer.java b/java/memory/src/main/java/org/apache/arrow/memory/util/ArrowBufPointer.java
new file mode 100644
index 0000000..cafe4b0
--- /dev/null
+++ b/java/memory/src/main/java/org/apache/arrow/memory/util/ArrowBufPointer.java
@@ -0,0 +1,106 @@
+/*
+ * 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.
+ */
+
+package org.apache.arrow.memory.util;
+
+import io.netty.buffer.ArrowBuf;
+
+/**
+ * Pointer to a memory region within an {@link io.netty.buffer.ArrowBuf}.
+ * It will be used as the basis for calculating hash code within a vector, and equality determination.
+ */
+public final class ArrowBufPointer {
+
+  private ArrowBuf buf;
+
+  private int offset;
+
+  private int length;
+
+  /**
+   * The default constructor.
+   */
+  public ArrowBufPointer() {
+
+  }
+
+  /**
+   * Constructs an Arrow buffer pointer.
+   * @param buf the underlying {@link ArrowBuf}, which can be null.
+   * @param offset the start off set of the memory region pointed to.
+   * @param length the length off set of the memory region pointed to.
+   */
+  public ArrowBufPointer(ArrowBuf buf, int offset, int length) {
+    set(buf, offset, length);
+  }
+
+  /**
+   * Sets this pointer.
+   * @param buf the underlying {@link ArrowBuf}, which can be null.
+   * @param offset the start off set of the memory region pointed to.
+   * @param length the length off set of the memory region pointed to.
+   */
+  public void set(ArrowBuf buf, int offset, int length) {
+    this.buf = buf;
+    this.offset = offset;
+    this.length = length;
+  }
+
+  /**
+   * Gets the underlying buffer, or null if the underlying data is invalid or null.
+   * @return the underlying buffer, if any, or null if the underlying data is invalid or null.
+   */
+  public ArrowBuf getBuf() {
+    return buf;
+  }
+
+  public int getOffset() {
+    return offset;
+  }
+
+  public int getLength() {
+    return length;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    ArrowBufPointer other = (ArrowBufPointer) o;
+    if (buf == null || other.buf == null) {
+      if (buf == null && other.buf == null) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    return ByteFunctionHelpers.equal(buf, offset, offset + length,
+            other.buf, other.offset, other.offset + other.length) != 0;
+  }
+
+  @Override
+  public int hashCode() {
+    // implement after ARROW-5898
+    throw new UnsupportedOperationException();
+  }
+}
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java b/java/memory/src/main/java/org/apache/arrow/memory/util/ByteFunctionHelpers.java
similarity index 99%
copy from java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java
copy to java/memory/src/main/java/org/apache/arrow/memory/util/ByteFunctionHelpers.java
index 8dbdc49..c2f83fc 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java
+++ b/java/memory/src/main/java/org/apache/arrow/memory/util/ByteFunctionHelpers.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.arrow.vector.util;
+package org.apache.arrow.memory.util;
 
 import org.apache.arrow.memory.BoundsChecking;
 
@@ -49,48 +49,6 @@ public class ByteFunctionHelpers {
     return memEqual(left.memoryAddress(), lStart, lEnd, right.memoryAddress(), rStart, rEnd);
   }
 
-  /**
-   * Compute hashCode with the given {@link ArrowBuf} and start/end index.
-   */
-  public static final int hash(final ArrowBuf buf, int start, int end) {
-    long addr = buf.memoryAddress();
-    int len = end - start;
-    long pos = addr + start;
-
-    int hash = 0;
-
-    while (len > 7) {
-      long value = PlatformDependent.getLong(pos);
-      hash = comebineHash(hash, Long.hashCode(value));
-
-      pos += 8;
-      len -= 8;
-    }
-
-    while (len > 3) {
-      int value = PlatformDependent.getInt(pos);
-      hash = comebineHash(hash, value);
-
-      pos += 4;
-      len -= 4;
-    }
-
-    while (len-- != 0) {
-      byte value = PlatformDependent.getByte(pos);
-      hash = comebineHash(hash, value);
-      pos ++;
-    }
-
-    return hash;
-  }
-
-  /**
-   * Generate a new hashCode with the given current hashCode and new hashCode.
-   */
-  public static int comebineHash(int currentHash, int newHash) {
-    return currentHash * 31 + newHash;
-  }
-
   private static int memEqual(final long laddr, int lStart, int lEnd, final long raddr, int rStart,
                                     final int rEnd) {
 
@@ -288,4 +246,45 @@ public class ByteFunctionHelpers {
     return lLen > rLen ? 1 : -1;
   }
 
+  /**
+   * Compute hashCode with the given {@link ArrowBuf} and start/end index.
+   */
+  public static final int hash(final ArrowBuf buf, int start, int end) {
+    long addr = buf.memoryAddress();
+    int len = end - start;
+    long pos = addr + start;
+
+    int hash = 0;
+
+    while (len > 7) {
+      long value = PlatformDependent.getLong(pos);
+      hash = comebineHash(hash, Long.hashCode(value));
+
+      pos += 8;
+      len -= 8;
+    }
+
+    while (len > 3) {
+      int value = PlatformDependent.getInt(pos);
+      hash = comebineHash(hash, value);
+
+      pos += 4;
+      len -= 4;
+    }
+
+    while (len-- != 0) {
+      byte value = PlatformDependent.getByte(pos);
+      hash = comebineHash(hash, value);
+      pos ++;
+    }
+
+    return hash;
+  }
+
+  /**
+   * Generate a new hashCode with the given current hashCode and new hashCode.
+   */
+  public static int comebineHash(int currentHash, int newHash) {
+    return currentHash * 31 + newHash;
+  }
 }
diff --git a/java/memory/src/test/java/org/apache/arrow/memory/util/TestArrowBufPointer.java b/java/memory/src/test/java/org/apache/arrow/memory/util/TestArrowBufPointer.java
new file mode 100644
index 0000000..f9d738e
--- /dev/null
+++ b/java/memory/src/test/java/org/apache/arrow/memory/util/TestArrowBufPointer.java
@@ -0,0 +1,70 @@
+/*
+ * 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.
+ */
+
+package org.apache.arrow.memory.util;
+
+import static junit.framework.TestCase.assertTrue;
+
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import io.netty.buffer.ArrowBuf;
+
+/**
+ * Test cases for {@link ArrowBufPointer}.
+ */
+public class TestArrowBufPointer {
+
+  private final int BUFFER_LENGTH = 1024;
+
+  private BufferAllocator allocator;
+
+  @Before
+  public void prepare() {
+    allocator = new RootAllocator(1024 * 1024);
+  }
+
+  @After
+  public void shutdown() {
+    allocator.close();
+  }
+
+  @Test
+  public void testArrowBufPointersEqual() {
+    try (ArrowBuf buf1 = allocator.buffer(BUFFER_LENGTH);
+    ArrowBuf buf2 = allocator.buffer(BUFFER_LENGTH)) {
+      for (int i = 0; i < BUFFER_LENGTH / 4; i++) {
+        buf1.setInt(i * 4, i * 1234);
+        buf2.setInt(i * 4, i * 1234);
+      }
+
+      ArrowBufPointer ptr1 = new ArrowBufPointer(null, 0, 100);
+      ArrowBufPointer ptr2 = new ArrowBufPointer(null, 100, 5032);
+      assertTrue(ptr1.equals(ptr2));
+      for (int i = 0; i < BUFFER_LENGTH / 4; i++) {
+        ptr1.set(buf1, i * 4, 4);
+        ptr2.set(buf2, i * 4, 4);
+        assertTrue(ptr1.equals(ptr2));
+      }
+    }
+  }
+}
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/util/TestByteFunctionHelpers.java b/java/memory/src/test/java/org/apache/arrow/memory/util/TestByteFunctionHelpers.java
similarity index 98%
rename from java/vector/src/test/java/org/apache/arrow/vector/util/TestByteFunctionHelpers.java
rename to java/memory/src/test/java/org/apache/arrow/memory/util/TestByteFunctionHelpers.java
index cb87927..f377278 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/util/TestByteFunctionHelpers.java
+++ b/java/memory/src/test/java/org/apache/arrow/memory/util/TestByteFunctionHelpers.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.arrow.vector.util;
+package org.apache.arrow.memory.util;
 
 import static org.junit.Assert.assertEquals;
 
diff --git a/java/performance/src/test/java/org/apache/arrow/vector/util/ByteFunctionHelpersBenchmarks.java b/java/performance/src/test/java/org/apache/arrow/memory/util/ByteFunctionHelpersBenchmarks.java
similarity index 98%
rename from java/performance/src/test/java/org/apache/arrow/vector/util/ByteFunctionHelpersBenchmarks.java
rename to java/performance/src/test/java/org/apache/arrow/memory/util/ByteFunctionHelpersBenchmarks.java
index 6f380a8..caee813 100644
--- a/java/performance/src/test/java/org/apache/arrow/vector/util/ByteFunctionHelpersBenchmarks.java
+++ b/java/performance/src/test/java/org/apache/arrow/memory/util/ByteFunctionHelpersBenchmarks.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.arrow.vector.util;
+package org.apache.arrow.memory.util;
 
 import java.util.concurrent.TimeUnit;
 
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java
index b0d716a..76228b4 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java
@@ -23,9 +23,10 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.util.ArrowBufPointer;
+import org.apache.arrow.memory.util.ByteFunctionHelpers;
 import org.apache.arrow.vector.ipc.message.ArrowFieldNode;
 import org.apache.arrow.vector.types.pojo.Field;
-import org.apache.arrow.vector.util.ByteFunctionHelpers;
 import org.apache.arrow.vector.util.CallBack;
 import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.util.TransferPair;
@@ -840,6 +841,21 @@ public abstract class BaseFixedWidthVector extends BaseValueVector
   }
 
   @Override
+  public ArrowBufPointer getDataPointer(int index) {
+    return getDataPointer(index, new ArrowBufPointer());
+  }
+
+  @Override
+  public ArrowBufPointer getDataPointer(int index, ArrowBufPointer reuse) {
+    if (isNull(index)) {
+      reuse.set(null, 0, 0);
+    } else {
+      reuse.set(valueBuffer, index * typeWidth, typeWidth);
+    }
+    return reuse;
+  }
+
+  @Override
   public int hashCode(int index) {
     int start = typeWidth * index;
     int end = typeWidth * (index + 1);
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java
index 19fcc67..4386e3b 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java
@@ -25,9 +25,10 @@ import java.util.List;
 import org.apache.arrow.memory.BaseAllocator;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.OutOfMemoryException;
+import org.apache.arrow.memory.util.ArrowBufPointer;
+import org.apache.arrow.memory.util.ByteFunctionHelpers;
 import org.apache.arrow.vector.ipc.message.ArrowFieldNode;
 import org.apache.arrow.vector.types.pojo.Field;
-import org.apache.arrow.vector.util.ByteFunctionHelpers;
 import org.apache.arrow.vector.util.CallBack;
 import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.util.TransferPair;
@@ -1337,6 +1338,23 @@ public abstract class BaseVariableWidthVector extends BaseValueVector
   }
 
   @Override
+  public ArrowBufPointer getDataPointer(int index) {
+    return getDataPointer(index, new ArrowBufPointer());
+  }
+
+  @Override
+  public ArrowBufPointer getDataPointer(int index, ArrowBufPointer reuse) {
+    if (isNull(index)) {
+      reuse.set(null, 0, 0);
+    } else {
+      int offset = offsetBuffer.getInt(index * OFFSET_WIDTH);
+      int length = offsetBuffer.getInt((index + 1) * OFFSET_WIDTH) - offset;
+      reuse.set(valueBuffer, offset, length);
+    }
+    return reuse;
+  }
+
+  @Override
   public int hashCode(int index) {
     final int start = getStartOffset(index);
     final int end = getStartOffset(index + 1);
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/FixedWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/FixedWidthVector.java
index d70ae29..2dee1f4 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/FixedWidthVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/FixedWidthVector.java
@@ -18,6 +18,8 @@
 package org.apache.arrow.vector;
 
 
+import org.apache.arrow.memory.util.ArrowBufPointer;
+
 /**
  * Interface for all fixed width {@link ValueVector} (e.g. integer, fixed size binary, etc).
  */
@@ -35,4 +37,18 @@ public interface FixedWidthVector extends ValueVector {
    */
   void zeroVector();
 
+  /**
+   * Gets the pointer for the data at the given index.
+   * @param index the index for the data.
+   * @return the pointer to the data.
+   */
+  ArrowBufPointer getDataPointer(int index);
+
+  /**
+   * Gets the pointer for the data at the given index.
+   * @param index the index for the data.
+   * @param reuse the data pointer to fill, this avoids creating a new pointer object.
+   * @return the pointer to the data, it should be the same one as the input parameter
+   */
+  ArrowBufPointer getDataPointer(int index, ArrowBufPointer reuse);
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/VariableWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/VariableWidthVector.java
index b1bcbc7..738ad85 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/VariableWidthVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/VariableWidthVector.java
@@ -17,6 +17,8 @@
 
 package org.apache.arrow.vector;
 
+import org.apache.arrow.memory.util.ArrowBufPointer;
+
 /**
  * Interface vectors that contain variable width members (e.g. Strings, Lists, etc).
  */
@@ -42,4 +44,19 @@ public interface VariableWidthVector extends ValueVector, DensityAwareVector {
    * @return the number of bytes in valueBuffer.
    */
   int sizeOfValueBuffer();
+
+  /**
+   * Gets the pointer for the data at the given index.
+   * @param index the index for the data.
+   * @return the pointer to the data.
+   */
+  ArrowBufPointer getDataPointer(int index);
+
+  /**
+   * Gets the pointer for the data at the given index.
+   * @param index the index for the data.
+   * @param reuse the data pointer to fill, this avoids creating a new pointer object.
+   * @return the pointer to the data, it should be the same one as the input parameter.
+   */
+  ArrowBufPointer getDataPointer(int index, ArrowBufPointer reuse);
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java
index c665bef..39d0287 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java
@@ -30,6 +30,7 @@ import java.util.Objects;
 import org.apache.arrow.memory.BaseAllocator;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.OutOfMemoryException;
+import org.apache.arrow.memory.util.ByteFunctionHelpers;
 import org.apache.arrow.util.Preconditions;
 import org.apache.arrow.vector.AddOrGetResult;
 import org.apache.arrow.vector.BaseValueVector;
@@ -45,7 +46,6 @@ import org.apache.arrow.vector.types.pojo.ArrowType;
 import org.apache.arrow.vector.types.pojo.DictionaryEncoding;
 import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.FieldType;
-import org.apache.arrow.vector.util.ByteFunctionHelpers;
 import org.apache.arrow.vector.util.CallBack;
 import org.apache.arrow.vector.util.JsonStringArrayList;
 import org.apache.arrow.vector.util.OversizedAllocationException;
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java b/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java
index 8dbdc49..ca3ab81 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java
@@ -17,21 +17,19 @@
 
 package org.apache.arrow.vector.util;
 
-import org.apache.arrow.memory.BoundsChecking;
-
 import io.netty.buffer.ArrowBuf;
-import io.netty.util.internal.PlatformDependent;
 
 /**
- * Utility methods for memory comparison at a byte level.
+ * @deprecated This class will be removed. Please use org.apache.arrow.memory.util.ByteFunctionHelpers instead.
  */
+@Deprecated
 public class ByteFunctionHelpers {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ByteFunctionHelpers.class);
 
   private ByteFunctionHelpers() {}
 
   /**
-   * Helper function to check for equality of bytes in two ArrowBufs.
+   * @deprecated Helper function to check for equality of bytes in two ArrowBufs.
    *
    * @param left   Left ArrowBuf for comparison
    * @param lStart start offset in the buffer
@@ -41,105 +39,14 @@ public class ByteFunctionHelpers {
    * @param rEnd   end offset in the buffer
    * @return 1 if equals, 0 otherwise
    */
+  @Deprecated
   public static final int equal(final ArrowBuf left, int lStart, int lEnd, final ArrowBuf right, int rStart, int rEnd) {
-    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
-      left.checkBytes(lStart, lEnd);
-      right.checkBytes(rStart, rEnd);
-    }
-    return memEqual(left.memoryAddress(), lStart, lEnd, right.memoryAddress(), rStart, rEnd);
-  }
-
-  /**
-   * Compute hashCode with the given {@link ArrowBuf} and start/end index.
-   */
-  public static final int hash(final ArrowBuf buf, int start, int end) {
-    long addr = buf.memoryAddress();
-    int len = end - start;
-    long pos = addr + start;
-
-    int hash = 0;
-
-    while (len > 7) {
-      long value = PlatformDependent.getLong(pos);
-      hash = comebineHash(hash, Long.hashCode(value));
-
-      pos += 8;
-      len -= 8;
-    }
-
-    while (len > 3) {
-      int value = PlatformDependent.getInt(pos);
-      hash = comebineHash(hash, value);
-
-      pos += 4;
-      len -= 4;
-    }
-
-    while (len-- != 0) {
-      byte value = PlatformDependent.getByte(pos);
-      hash = comebineHash(hash, value);
-      pos ++;
-    }
-
-    return hash;
+    return org.apache.arrow.memory.util.ByteFunctionHelpers.equal(left, lStart, lEnd, right, rStart, rEnd);
   }
 
   /**
-   * Generate a new hashCode with the given current hashCode and new hashCode.
-   */
-  public static int comebineHash(int currentHash, int newHash) {
-    return currentHash * 31 + newHash;
-  }
-
-  private static int memEqual(final long laddr, int lStart, int lEnd, final long raddr, int rStart,
-                                    final int rEnd) {
-
-    int n = lEnd - lStart;
-    if (n == rEnd - rStart) {
-      long lPos = laddr + lStart;
-      long rPos = raddr + rStart;
-
-      while (n > 7) {
-        long leftLong = PlatformDependent.getLong(lPos);
-        long rightLong = PlatformDependent.getLong(rPos);
-        if (leftLong != rightLong) {
-          return 0;
-        }
-        lPos += 8;
-        rPos += 8;
-        n -= 8;
-      }
-
-      while (n > 3) {
-        int leftInt = PlatformDependent.getInt(lPos);
-        int rightInt = PlatformDependent.getInt(rPos);
-        if (leftInt != rightInt) {
-          return 0;
-        }
-        lPos += 4;
-        rPos += 4;
-        n -= 4;
-      }
-
-      while (n-- != 0) {
-        byte leftByte = PlatformDependent.getByte(lPos);
-        byte rightByte = PlatformDependent.getByte(rPos);
-        if (leftByte != rightByte) {
-          return 0;
-        }
-        lPos++;
-        rPos++;
-      }
-      return 1;
-    } else {
-      return 0;
-    }
-  }
-
-  /**
-   * Helper function to compare a set of bytes in two ArrowBufs.
-   *
-   * <p>Function will check data before completing in the case that
+   * @deprecated Helper function to compare a set of bytes in two ArrowBufs.
+   *     Function will check data before completing in the case that
    *
    * @param left   Left ArrowBuf to compare
    * @param lStart start offset in the buffer
@@ -149,6 +56,7 @@ public class ByteFunctionHelpers {
    * @param rEnd   end offset in the buffer
    * @return 1 if left input is greater, -1 if left input is smaller, 0 otherwise
    */
+  @Deprecated
   public static final int compare(
       final ArrowBuf left,
       int lStart,
@@ -156,68 +64,11 @@ public class ByteFunctionHelpers {
       final ArrowBuf right,
       int rStart,
       int rEnd) {
-    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
-      left.checkBytes(lStart, lEnd);
-      right.checkBytes(rStart, rEnd);
-    }
-    return memcmp(left.memoryAddress(), lStart, lEnd, right.memoryAddress(), rStart, rEnd);
-  }
-
-  private static int memcmp(
-      final long laddr,
-      int lStart,
-      int lEnd,
-      final long raddr,
-      int rStart,
-      final int rEnd) {
-    int lLen = lEnd - lStart;
-    int rLen = rEnd - rStart;
-    int n = Math.min(rLen, lLen);
-    long lPos = laddr + lStart;
-    long rPos = raddr + rStart;
-
-    while (n > 7) {
-      long leftLong = PlatformDependent.getLong(lPos);
-      long rightLong = PlatformDependent.getLong(rPos);
-      if (leftLong != rightLong) {
-        return unsignedLongCompare(leftLong, rightLong);
-      }
-      lPos += 8;
-      rPos += 8;
-      n -= 8;
-    }
-
-    while (n > 3) {
-      int leftInt = PlatformDependent.getInt(lPos);
-      int rightInt = PlatformDependent.getInt(rPos);
-      if (leftInt != rightInt) {
-        return unsignedIntCompare(leftInt, rightInt);
-      }
-      lPos += 4;
-      rPos += 4;
-      n -= 4;
-    }
-
-    while (n-- != 0) {
-      byte leftByte = PlatformDependent.getByte(lPos);
-      byte rightByte = PlatformDependent.getByte(rPos);
-      if (leftByte != rightByte) {
-        return ((leftByte & 0xFF) - (rightByte & 0xFF)) > 0 ? 1 : -1;
-      }
-      lPos++;
-      rPos++;
-    }
-
-    if (lLen == rLen) {
-      return 0;
-    }
-
-    return lLen > rLen ? 1 : -1;
-
+    return org.apache.arrow.memory.util.ByteFunctionHelpers.compare(left, lStart, lEnd, right, rStart, rEnd);
   }
 
   /**
-   * Helper function to compare a set of bytes in ArrowBuf to a ByteArray.
+   * @deprecated Helper function to compare a set of bytes in ArrowBuf to a ByteArray.
    *
    * @param left   Left ArrowBuf for comparison purposes
    * @param lStart start offset in the buffer
@@ -227,6 +78,7 @@ public class ByteFunctionHelpers {
    * @param rEnd   end offset in the byte array
    * @return 1 if left input is greater, -1 if left input is smaller, 0 otherwise
    */
+  @Deprecated
   public static final int compare(
       final ArrowBuf left,
       int lStart,
@@ -234,15 +86,11 @@ public class ByteFunctionHelpers {
       final byte[] right,
       int rStart,
       final int rEnd) {
-    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
-      left.checkBytes(lStart, lEnd);
-    }
-    return memcmp(left.memoryAddress(), lStart, lEnd, right, rStart, rEnd);
+    return org.apache.arrow.memory.util.ByteFunctionHelpers.compare(left, lStart, lEnd, right, rStart, rEnd);
   }
 
-
   /**
-   * Compares the two specified {@code long} values, treating them as unsigned values between
+   * @deprecated Compares the two specified {@code long} values, treating them as unsigned values between
    * {@code 0} and {@code 2^64 - 1} inclusive.
    *
    * @param a the first unsigned {@code long} to compare
@@ -250,42 +98,13 @@ public class ByteFunctionHelpers {
    * @return a negative value if {@code a} is less than {@code b}; a positive value if {@code a} is
    *     greater than {@code b}; or zero if they are equal
    */
+  @Deprecated
   public static int unsignedLongCompare(long a, long b) {
-    return Long.compare(a ^ Long.MIN_VALUE, b ^ Long.MIN_VALUE);
+    return org.apache.arrow.memory.util.ByteFunctionHelpers.unsignedLongCompare(a, b);
   }
 
+  @Deprecated
   public static int unsignedIntCompare(int a, int b) {
-    return Integer.compare(a ^ Integer.MIN_VALUE, b ^ Integer.MIN_VALUE);
+    return org.apache.arrow.memory.util.ByteFunctionHelpers.unsignedIntCompare(a, b);
   }
-
-  private static int memcmp(
-      final long laddr,
-      int lStart,
-      int lEnd,
-      final byte[] right,
-      int rStart,
-      final int rEnd) {
-    int lLen = lEnd - lStart;
-    int rLen = rEnd - rStart;
-    int n = Math.min(rLen, lLen);
-    long lPos = laddr + lStart;
-    int rPos = rStart;
-
-    while (n-- != 0) {
-      byte leftByte = PlatformDependent.getByte(lPos);
-      byte rightByte = right[rPos];
-      if (leftByte != rightByte) {
-        return ((leftByte & 0xFF) - (rightByte & 0xFF)) > 0 ? 1 : -1;
-      }
-      lPos++;
-      rPos++;
-    }
-
-    if (lLen == rLen) {
-      return 0;
-    }
-
-    return lLen > rLen ? 1 : -1;
-  }
-
 }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
index f8f9c76..c38f4dc 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
@@ -35,6 +35,7 @@ import java.util.List;
 import org.apache.arrow.memory.BaseAllocator;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.memory.util.ArrowBufPointer;
 import org.apache.arrow.vector.holders.NullableVarBinaryHolder;
 import org.apache.arrow.vector.holders.NullableVarCharHolder;
 import org.apache.arrow.vector.ipc.message.ArrowRecordBatch;
@@ -2162,6 +2163,75 @@ public class TestValueVector {
   }
 
   @Test
+  public void testGetPointerFixedWidth() {
+    final int vectorLength = 100;
+    try (IntVector vec1 = new IntVector("vec1", allocator);
+         IntVector vec2 = new IntVector("vec2", allocator)) {
+      vec1.allocateNew(vectorLength);
+      vec2.allocateNew(vectorLength);
+
+      for (int i = 0; i < vectorLength; i++) {
+        if (i % 10 == 0) {
+          vec1.setNull(i);
+          vec2.setNull(i);
+        } else {
+          vec1.set(i, i * 1234);
+          vec2.set(i, i * 1234);
+        }
+      }
+
+      ArrowBufPointer ptr1 = new ArrowBufPointer();
+      ArrowBufPointer ptr2 = new ArrowBufPointer();
+
+      for (int i = 0; i < vectorLength; i++) {
+        vec1.getDataPointer(i, ptr1);
+        vec2.getDataPointer(i, ptr2);
+
+        if (i % 10 == 0) {
+          assertNull(ptr1.getBuf());
+          assertNull(ptr2.getBuf());
+        }
+
+        assertTrue(ptr1.equals(ptr2));
+        assertTrue(ptr2.equals(ptr2));
+      }
+    }
+  }
+
+  @Test
+  public void testGetPointerVariableWidth() {
+    final String[] sampleData = new String[]{
+      "abc", "123", "def", null, "hello", "aaaaa", "world", "2019", null, "0717"};
+
+    try (VarCharVector vec1 = new VarCharVector("vec1", allocator);
+         VarCharVector vec2 = new VarCharVector("vec2", allocator)) {
+      vec1.allocateNew(sampleData.length * 10, sampleData.length);
+      vec2.allocateNew(sampleData.length * 10, sampleData.length);
+
+      for (int i = 0; i < sampleData.length; i++) {
+        String str = sampleData[i];
+        if (str != null) {
+          vec1.set(i, sampleData[i].getBytes());
+          vec2.set(i, sampleData[i].getBytes());
+        } else {
+          vec1.setNull(i);
+          vec2.setNull(i);
+        }
+      }
+
+      ArrowBufPointer ptr1 = new ArrowBufPointer();
+      ArrowBufPointer ptr2 = new ArrowBufPointer();
+
+      for (int i = 0; i < sampleData.length; i++) {
+        vec1.getDataPointer(i, ptr1);
+        vec2.getDataPointer(i, ptr2);
+
+        assertTrue(ptr1.equals(ptr2));
+        assertTrue(ptr2.equals(ptr2));
+      }
+    }
+  }
+
   public void testGetNullFromVariableWidthVector() {
     try (VarCharVector varCharVector = new VarCharVector("varcharvec", allocator);
     VarBinaryVector varBinaryVector = new VarBinaryVector("varbinary", allocator)) {