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/05 17:25:56 UTC

[arrow] branch master updated: ARROW-5834: [Java] Apply new hash map in DictionaryEncoder

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 c037ee2  ARROW-5834: [Java] Apply new hash map in DictionaryEncoder
c037ee2 is described below

commit c037ee2be7c8065032494ba7e052941edaf3ac61
Author: tianchen <ni...@alibaba-inc.com>
AuthorDate: Fri Jul 5 10:25:26 2019 -0700

    ARROW-5834: [Java] Apply new hash map in DictionaryEncoder
    
    Related to [ARROW-5834](https://issues.apache.org/jira/browse/ARROW-5834).
    Apply new hash map in DictionaryEncoder to make it work.
    Meanwhile provide benchmark for DictionaryEncoder:
    
    DictionaryEncoderBenchmarks#testEncode:
    Before: 5  430860.989 ± 14157.675  ns/op
    After:   5  415703.943 ± 9258.049  ns/op
    
    Author: tianchen <ni...@alibaba-inc.com>
    
    Closes #4786 from tianchen92/ARROW-5834 and squashes the following commits:
    
    dce8b6ae6 <tianchen> fix
    43f7695a2 <tianchen> Apply new hash map in DictionaryEncoder
---
 .../dictionary/DictionaryEncoderBenchmarks.java    | 149 +++++++++++++++++++++
 .../arrow/vector/dictionary/DictionaryEncoder.java |   9 +-
 2 files changed, 152 insertions(+), 6 deletions(-)

diff --git a/java/performance/src/test/java/org/apache/arrow/vector/dictionary/DictionaryEncoderBenchmarks.java b/java/performance/src/test/java/org/apache/arrow/vector/dictionary/DictionaryEncoderBenchmarks.java
new file mode 100644
index 0000000..047807c
--- /dev/null
+++ b/java/performance/src/test/java/org/apache/arrow/vector/dictionary/DictionaryEncoderBenchmarks.java
@@ -0,0 +1,149 @@
+/*
+ * 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.dictionary;
+
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.types.pojo.DictionaryEncoding;
+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;
+
+/**
+ * Benchmarks for {@link DictionaryEncoder}.
+ */
+@State(Scope.Benchmark)
+public class DictionaryEncoderBenchmarks {
+
+  private BufferAllocator allocator;
+
+  private static final int DATA_SIZE = 1000;
+  private static final int KEY_SIZE = 100;
+
+
+  private static final int KEY_LENGTH = 10;
+
+  private List<String> keys = new ArrayList<>();
+
+  private VarCharVector vector;
+
+  private VarCharVector dictionaryVector;
+
+  /**
+   * Setup benchmarks.
+   */
+  @Setup
+  public void prepare() {
+
+    for (int i = 0; i < KEY_SIZE; i++) {
+      keys.add(generateUniqueKey(KEY_LENGTH));
+    }
+
+    allocator = new RootAllocator(10 * 1024 * 1024);
+
+    vector = new VarCharVector("vector", allocator);
+    dictionaryVector = new VarCharVector("dict", allocator);
+
+    vector.allocateNew(10240, DATA_SIZE);
+    vector.setValueCount(DATA_SIZE);
+    for (int i = 0; i < DATA_SIZE; i++) {
+      byte[] value = keys.get(generateRandomIndex(KEY_SIZE)).getBytes(StandardCharsets.UTF_8);
+      vector.setSafe(i, value, 0, value.length);
+    }
+
+    dictionaryVector.allocateNew(1024, 100);
+    dictionaryVector.setValueCount(100);
+    for (int i = 0; i < KEY_SIZE; i++) {
+      byte[] value = keys.get(i).getBytes(StandardCharsets.UTF_8);
+      dictionaryVector.setSafe(i, value, 0, value.length);
+    }
+
+  }
+
+  /**
+   * Tear down benchmarks.
+   */
+  @TearDown
+  public void tearDown() {
+    vector.close();
+    dictionaryVector.close();
+    keys.clear();
+    allocator.close();
+  }
+
+  /**
+   * Test encode for {@link DictionaryEncoder}.
+   * @return useless. To avoid DCE by JIT.
+   */
+  @Benchmark
+  @BenchmarkMode(Mode.AverageTime)
+  @OutputTimeUnit(TimeUnit.NANOSECONDS)
+  public int testEncode() {
+    Dictionary dictionary = new Dictionary(dictionaryVector, new DictionaryEncoding(1L, false, null));
+    final ValueVector encoded = DictionaryEncoder.encode(vector, dictionary);
+    encoded.close();
+    return 0;
+  }
+
+  private int generateRandomIndex(int max) {
+    Random random = new Random();
+    return random.nextInt(max);
+  }
+
+  private String generateUniqueKey(int length) {
+    String str = "abcdefghijklmnopqrstuvwxyz";
+    Random random = new Random();
+    StringBuffer sb = new StringBuffer();
+    for (int i = 0; i < length; i++) {
+      int number = random.nextInt(26);
+      sb.append(str.charAt(number));
+    }
+    if (keys.contains(sb.toString())) {
+      return generateUniqueKey(length);
+    }
+    return sb.toString();
+  }
+
+  @Test
+  public void evaluate() throws RunnerException {
+    Options opt = new OptionsBuilder()
+        .include(DictionaryEncoderBenchmarks.class.getSimpleName())
+        .forks(1)
+        .build();
+
+    new Runner(opt).run();
+  }
+}
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryEncoder.java b/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryEncoder.java
index b9f547c..ccd4b55 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryEncoder.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryEncoder.java
@@ -17,9 +17,6 @@
 
 package org.apache.arrow.vector.dictionary;
 
-import java.util.HashMap;
-import java.util.Map;
-
 import org.apache.arrow.vector.BaseIntVector;
 import org.apache.arrow.vector.FieldVector;
 import org.apache.arrow.vector.ValueVector;
@@ -47,7 +44,7 @@ public class DictionaryEncoder {
   public static ValueVector encode(ValueVector vector, Dictionary dictionary) {
     validateType(vector.getMinorType());
     // load dictionary values into a hashmap for lookup
-    Map<Object, Integer> lookUps = new HashMap<>(dictionary.getVector().getValueCount());
+    DictionaryEncodeHashMap<Object> lookUps = new DictionaryEncodeHashMap<>(dictionary.getVector().getValueCount());
     for (int i = 0; i < dictionary.getVector().getValueCount(); i++) {
       // for primitive array types we need a wrapper that implements equals and hashcode appropriately
       lookUps.put(dictionary.getVector().getObject(i), i);
@@ -74,8 +71,8 @@ public class DictionaryEncoder {
       Object value = vector.getObject(i);
       if (value != null) { // if it's null leave it null
         // note: this may fail if value was not included in the dictionary
-        Integer encoded = lookUps.get(value);
-        if (encoded == null) {
+        int encoded = lookUps.get(value);
+        if (encoded == -1) {
           throw new IllegalArgumentException("Dictionary encoding not defined for value:" + value);
         }
         indices.setWithPossibleTruncate(i, encoded);