mirror of https://github.com/apache/druid.git
perf: indexing: Introduce a bulk getValuesInto function to read values (#12105)
* perf: indexing: Introduce a bulk getValuesInto function to read values in bulk If large number of values are required from DimensionDictionary during indexing, fetch them all in a single lock/unlock instead of lock/unlock each individual item. * refactor: rename key to keys in function args * fix: check explicitly that argument length on arrays match * refactor: getValuesInto renamed to getValues, now creates and returns a new T[] rather than filling
This commit is contained in:
parent
b86f2d4c2e
commit
eb1b53b7f8
|
@ -32,6 +32,7 @@ import org.openjdk.jmh.annotations.Param;
|
|||
import org.openjdk.jmh.annotations.Scope;
|
||||
import org.openjdk.jmh.annotations.Setup;
|
||||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.annotations.Threads;
|
||||
import org.openjdk.jmh.annotations.Warmup;
|
||||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
|
@ -53,7 +54,7 @@ public class StringDimensionIndexerBenchmark
|
|||
@Param({"10000"})
|
||||
public int cardinality;
|
||||
|
||||
@Param({"8"})
|
||||
@Param({"8", "40"})
|
||||
public int rowSize;
|
||||
|
||||
@Setup
|
||||
|
@ -75,9 +76,20 @@ public class StringDimensionIndexerBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
@Threads(1)
|
||||
public void estimateEncodedKeyComponentSize(Blackhole blackhole)
|
||||
{
|
||||
long sz = indexer.estimateEncodedKeyComponentSize(exampleArray);
|
||||
blackhole.consume(sz);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
@Threads(2)
|
||||
public void estimateEncodedKeyComponentSizeTwoThreads(Blackhole blackhole)
|
||||
{
|
||||
long sz = indexer.estimateEncodedKeyComponentSize(exampleArray);
|
||||
blackhole.consume(sz);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -45,17 +45,6 @@ public abstract class DictionaryEncodedColumnIndexer<KeyType, ActualType extends
|
|||
@Nullable
|
||||
protected SortedDimensionDictionary<ActualType> sortedLookup;
|
||||
|
||||
/**
|
||||
* Creates a new DictionaryEncodedColumnIndexer with the default implementation
|
||||
* of {@link DimensionDictionary}.
|
||||
* <p>
|
||||
* Using this constructor disables memory estimations of the dictionary size.
|
||||
*/
|
||||
public DictionaryEncodedColumnIndexer()
|
||||
{
|
||||
this(new DimensionDictionary<>());
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new DictionaryEncodedColumnIndexer.
|
||||
*
|
||||
|
|
|
@ -23,6 +23,8 @@ import it.unimi.dsi.fastutil.objects.Object2IntMap;
|
|||
import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.lang.reflect.Array;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -38,6 +40,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
public class DimensionDictionary<T extends Comparable<T>>
|
||||
{
|
||||
public static final int ABSENT_VALUE_ID = -1;
|
||||
private final Class<T> cls;
|
||||
|
||||
@Nullable
|
||||
private T minValue = null;
|
||||
|
@ -51,8 +54,9 @@ public class DimensionDictionary<T extends Comparable<T>>
|
|||
private final List<T> idToValue = new ArrayList<>();
|
||||
private final ReentrantReadWriteLock lock;
|
||||
|
||||
public DimensionDictionary()
|
||||
public DimensionDictionary(Class<T> cls)
|
||||
{
|
||||
this.cls = cls;
|
||||
this.lock = new ReentrantReadWriteLock();
|
||||
valueToId.defaultReturnValue(ABSENT_VALUE_ID);
|
||||
}
|
||||
|
@ -86,6 +90,22 @@ public class DimensionDictionary<T extends Comparable<T>>
|
|||
}
|
||||
}
|
||||
|
||||
public T[] getValues(int[] ids)
|
||||
{
|
||||
T[] values = (T[]) Array.newInstance(cls, ids.length);
|
||||
|
||||
lock.readLock().lock();
|
||||
try {
|
||||
for (int i = 0; i < ids.length; i++) {
|
||||
values[i] = (ids[i] == idForNull) ? null : idToValue.get(ids[i]);
|
||||
}
|
||||
return values;
|
||||
}
|
||||
finally {
|
||||
lock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
public int size()
|
||||
{
|
||||
lock.readLock().lock();
|
||||
|
|
|
@ -34,6 +34,7 @@ public class StringDimensionDictionary extends DimensionDictionary<String>
|
|||
*/
|
||||
public StringDimensionDictionary(boolean computeOnHeapSize)
|
||||
{
|
||||
super(String.class);
|
||||
this.computeOnHeapSize = computeOnHeapSize;
|
||||
}
|
||||
|
||||
|
|
|
@ -148,14 +148,15 @@ public class StringDimensionIndexer extends DictionaryEncodedColumnIndexer<int[]
|
|||
* Deprecated method. Use {@link #processRowValsToUnsortedEncodedKeyComponent(Object, boolean)}
|
||||
* and {@link EncodedKeyComponent#getEffectiveSizeBytes()}.
|
||||
*/
|
||||
public long estimateEncodedKeyComponentSize(int[] key)
|
||||
public long estimateEncodedKeyComponentSize(int[] keys)
|
||||
{
|
||||
// string length is being accounted for each time they are referenced, based on dimension handler interface,
|
||||
// even though they are stored just once. It may overestimate the size by a bit, but we wanted to leave
|
||||
// more buffer to be safe
|
||||
long estimatedSize = key.length * Integer.BYTES;
|
||||
for (int element : key) {
|
||||
String val = dimLookup.getValue(element);
|
||||
long estimatedSize = keys.length * Integer.BYTES;
|
||||
|
||||
String[] vals = dimLookup.getValues(keys);
|
||||
for (String val : vals) {
|
||||
if (val != null) {
|
||||
// According to https://www.ibm.com/developerworks/java/library/j-codetoheap/index.html
|
||||
// String has the following memory usuage...
|
||||
|
|
Loading…
Reference in New Issue