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

[arrow] branch master updated: ARROW-5909: [Java] Optimize ByteFunctionHelpers equals & compare logic

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

emkornfield 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 3b79a20  ARROW-5909: [Java] Optimize ByteFunctionHelpers equals & compare logic
3b79a20 is described below

commit 3b79a207f7c1b4b17695e56f08efcdaf0f5f93cd
Author: tianchen <ni...@alibaba-inc.com>
AuthorDate: Fri Jul 12 23:53:00 2019 -0700

    ARROW-5909: [Java] Optimize ByteFunctionHelpers equals & compare logic
    
    Related to [ARROW-5909](https://issues.apache.org/jira/browse/ARROW-5909).
    Now it first compare Long values and then if length < 8 then it compares Byte values.
    Add the logic to compare Int values when 4 < length < 8.
    
    Buffer length far greater than 8 have similar performance in equals,  in case buffer length < 8 performances are as below:
    
    ByteFunctionHelpersBenchmarks#equals
    Before: avgt    5  7.718 ± 0.401  ns/op
    After:    avgt    5  5.005 ± 0.830  ns/op
    
    Author: tianchen <ni...@alibaba-inc.com>
    
    Closes #4852 from tianchen92/ARROW-5909 and squashes the following commits:
    
    18db45fd3 <tianchen> add more UT
    dbd092365 <tianchen> remove reverse in ByteFunctionHelpers
    41030c4e9 <tianchen> fix
    e6ab2ed1d <tianchen> ARROW-5909:  Optimize ByteFunctionHelpers equals & compare logic
---
 .../vector/util/ByteFunctionHelpersBenchmarks.java |  98 ++++++++++++++++++
 .../arrow/vector/util/ByteFunctionHelpers.java     |  30 +++++-
 .../arrow/vector/util/TestByteFunctionHelpers.java | 114 +++++++++++++++++++++
 3 files changed, 240 insertions(+), 2 deletions(-)

diff --git a/java/performance/src/test/java/org/apache/arrow/vector/util/ByteFunctionHelpersBenchmarks.java b/java/performance/src/test/java/org/apache/arrow/vector/util/ByteFunctionHelpersBenchmarks.java
new file mode 100644
index 0000000..6f380a8
--- /dev/null
+++ b/java/performance/src/test/java/org/apache/arrow/vector/util/ByteFunctionHelpersBenchmarks.java
@@ -0,0 +1,98 @@
+/*
+ * 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.vector.util;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.junit.Test;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.TearDown;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+import io.netty.buffer.ArrowBuf;
+
+/**
+ * Benchmarks for {@link ByteFunctionHelpers}.
+ */
+@State(Scope.Benchmark)
+public class ByteFunctionHelpersBenchmarks {
+
+  private static final int BUFFER_CAPACITY = 7;
+
+  private static final int ALLOCATOR_CAPACITY = 1024 * 1024;
+
+  private BufferAllocator allocator;
+
+  private ArrowBuf buffer1;
+
+  private ArrowBuf buffer2;
+
+  /**
+   * Setup benchmarks.
+   */
+  @Setup
+  public void prepare() {
+    allocator = new RootAllocator(ALLOCATOR_CAPACITY);
+    buffer1 = allocator.buffer(BUFFER_CAPACITY);
+    buffer2 = allocator.buffer(BUFFER_CAPACITY);
+
+    for (int i = 0; i < BUFFER_CAPACITY; i++) {
+      buffer1.setByte(i, i);
+      buffer2.setByte(i, i);
+    }
+  }
+
+  /**
+   * Tear down benchmarks.
+   */
+  @TearDown
+  public void tearDown() {
+    buffer1.close();
+    buffer2.close();
+    allocator.close();
+  }
+
+  @Benchmark
+  @BenchmarkMode(Mode.AverageTime)
+  @OutputTimeUnit(TimeUnit.NANOSECONDS)
+  public void equals() {
+    ByteFunctionHelpers.equal(buffer1, 0, BUFFER_CAPACITY - 1, buffer2, 0, BUFFER_CAPACITY - 1);
+
+  }
+
+  @Test
+  public void evaluate() throws RunnerException {
+    Options opt = new OptionsBuilder()
+        .include(ByteFunctionHelpersBenchmarks.class.getSimpleName())
+        .forks(1)
+        .build();
+
+    new Runner(opt).run();
+  }
+}
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 8140103..fba32b6 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
@@ -39,7 +39,7 @@ public class ByteFunctionHelpers {
    * @param right  Right ArrowBuf for comparison
    * @param rStart start offset in the buffer
    * @param rEnd   end offset in the buffer
-   * @return 1 if left input is greater, -1 if left input is smaller, 0 otherwise
+   * @return 1 if equals, 0 otherwise
    */
   public static final int equal(final ArrowBuf left, int lStart, int lEnd, final ArrowBuf right, int rStart, int rEnd) {
     if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
@@ -67,6 +67,18 @@ public class ByteFunctionHelpers {
         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);
@@ -126,13 +138,24 @@ public class ByteFunctionHelpers {
       long leftLong = PlatformDependent.getLong(lPos);
       long rightLong = PlatformDependent.getLong(rPos);
       if (leftLong != rightLong) {
-        return unsignedLongCompare(Long.reverseBytes(leftLong), Long.reverseBytes(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);
@@ -189,6 +212,9 @@ public class ByteFunctionHelpers {
     return Long.compare(a ^ Long.MIN_VALUE, b ^ Long.MIN_VALUE);
   }
 
+  public static int unsignedIntCompare(int a, int b) {
+    return Integer.compare(a ^ Integer.MIN_VALUE, b ^ Integer.MIN_VALUE);
+  }
 
   private static int memcmp(
       final long laddr,
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/util/TestByteFunctionHelpers.java b/java/vector/src/test/java/org/apache/arrow/vector/util/TestByteFunctionHelpers.java
new file mode 100644
index 0000000..cb87927
--- /dev/null
+++ b/java/vector/src/test/java/org/apache/arrow/vector/util/TestByteFunctionHelpers.java
@@ -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.
+ */
+
+package org.apache.arrow.vector.util;
+
+import static org.junit.Assert.assertEquals;
+
+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;
+
+public class TestByteFunctionHelpers {
+
+  private BufferAllocator allocator;
+
+  private static final int SIZE = 100;
+
+  @Before
+  public void init() {
+    allocator = new RootAllocator(Long.MAX_VALUE);
+
+  }
+
+  @After
+  public void terminate() throws Exception {
+    allocator.close();
+  }
+
+  @Test
+  public void testEquals() {
+    ArrowBuf buffer1 = allocator.buffer(SIZE);
+    ArrowBuf buffer2 = allocator.buffer(SIZE);
+
+    for (int i = 0; i < SIZE; i++) {
+      buffer1.setByte(i, i);
+      buffer2.setByte(i, i);
+    }
+
+    //test three cases, length>8, length>3, length<3
+
+    assertEquals(1, ByteFunctionHelpers.equal(buffer1, 0, SIZE - 1,
+        buffer2, 0, SIZE - 1));
+    assertEquals(1, ByteFunctionHelpers.equal(buffer1, 0, 6,
+        buffer2, 0, 6));
+    assertEquals(1, ByteFunctionHelpers.equal(buffer1, 0, 2,
+        buffer2, 0, 2));
+
+    //change value at index1
+    buffer1.setByte(1, 10);
+
+    assertEquals(0, ByteFunctionHelpers.equal(buffer1, 0, SIZE - 1,
+        buffer2, 0, SIZE - 1));
+    assertEquals(0, ByteFunctionHelpers.equal(buffer1, 0, 6,
+        buffer2, 0, 6));
+    assertEquals(0, ByteFunctionHelpers.equal(buffer1, 0, 2,
+        buffer2, 0, 2));
+
+    buffer1.close();
+    buffer2.close();
+
+  }
+
+  @Test
+  public void testCompare() {
+    ArrowBuf buffer1 = allocator.buffer(SIZE);
+    ArrowBuf buffer2 = allocator.buffer(SIZE);
+
+    for (int i = 0; i < SIZE; i++) {
+      buffer1.setByte(i, i);
+      buffer2.setByte(i, i);
+    }
+
+    //test three cases, length>8, length>3, length<3
+
+    assertEquals(0, ByteFunctionHelpers.compare(buffer1, 0, SIZE - 1,
+        buffer2, 0, SIZE - 1));
+    assertEquals(0, ByteFunctionHelpers.compare(buffer1, 0, 6,
+        buffer2, 0, 6));
+    assertEquals(0, ByteFunctionHelpers.compare(buffer1, 0, 2,
+        buffer2, 0, 2));
+
+    //change value at index 1
+    buffer1.setByte(1, 0);
+
+    assertEquals(-1, ByteFunctionHelpers.compare(buffer1, 0, SIZE - 1,
+        buffer2, 0, SIZE - 1));
+    assertEquals(-1, ByteFunctionHelpers.compare(buffer1, 0, 6,
+        buffer2, 0, 6));
+    assertEquals(-1, ByteFunctionHelpers.compare(buffer1, 0, 2,
+        buffer2, 0, 2));
+
+    buffer1.close();
+    buffer2.close();
+
+  }
+}