RowBasedIndexedTable: Add specialized index types for long keys. (#10430)

* RowBasedIndexedTable: Add specialized index types for long keys.

Two new index types are added:

1) Use an int-array-based index in cases where the difference between
   the min and max values isn't too large, and keys are unique.

2) Use a Long2ObjectOpenHashMap (instead of the prior Java HashMap) in
   all other cases.

In addition:

1) RowBasedIndexBuilder, a new class, is responsible for picking which
   index implementation to use.

2) The IndexedTable.Index interface is extended to support using
   unboxed primitives in the unique-long-keys case, and callers are
   updated to use the new functionality.

Other key types continue to use indexes backed by Java HashMaps.

* Fixup logic.

* Add tests.
This commit is contained in:
Gian Merlino 2020-09-29 10:46:47 -07:00 committed by GitHub
parent 8168e14e92
commit 2be1ae128f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 831 additions and 161 deletions

View File

@ -20,8 +20,6 @@
package org.apache.druid.segment.join.table; package org.apache.druid.segment.join.table;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import it.unimi.dsi.fastutil.ints.IntArrayList;
import it.unimi.dsi.fastutil.ints.IntList;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequence;
@ -52,7 +50,6 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -67,9 +64,13 @@ public class BroadcastSegmentIndexedTable implements IndexedTable
private final Set<String> keyColumns; private final Set<String> keyColumns;
private final RowSignature rowSignature; private final RowSignature rowSignature;
private final String version; private final String version;
private final List<Map<Object, IntList>> keyColumnsIndex; private final List<Index> keyColumnsIndexes;
public BroadcastSegmentIndexedTable(final QueryableIndexSegment theSegment, final Set<String> keyColumns, final String version) public BroadcastSegmentIndexedTable(
final QueryableIndexSegment theSegment,
final Set<String> keyColumns,
final String version
)
{ {
this.keyColumns = keyColumns; this.keyColumns = keyColumns;
this.version = version; this.version = version;
@ -92,19 +93,22 @@ public class BroadcastSegmentIndexedTable implements IndexedTable
} }
this.rowSignature = sigBuilder.build(); this.rowSignature = sigBuilder.build();
// initialize keycolumn index maps // initialize keycolumn index builders
this.keyColumnsIndex = new ArrayList<>(rowSignature.size()); final ArrayList<RowBasedIndexBuilder> indexBuilders = new ArrayList<>(rowSignature.size());
final List<String> keyColumnNames = new ArrayList<>(keyColumns.size()); final List<String> keyColumnNames = new ArrayList<>(keyColumns.size());
for (int i = 0; i < rowSignature.size(); i++) { for (int i = 0; i < rowSignature.size(); i++) {
final Map<Object, IntList> m; final RowBasedIndexBuilder m;
final String columnName = rowSignature.getColumnName(i); final String columnName = rowSignature.getColumnName(i);
if (keyColumns.contains(columnName)) { if (keyColumns.contains(columnName)) {
m = new HashMap<>(); final ValueType keyType =
rowSignature.getColumnType(i).orElse(IndexedTableJoinMatcher.DEFAULT_KEY_TYPE);
m = new RowBasedIndexBuilder(keyType);
keyColumnNames.add(columnName); keyColumnNames.add(columnName);
} else { } else {
m = null; m = null;
} }
keyColumnsIndex.add(m); indexBuilders.add(m);
} }
// sort of like the dump segment tool, but build key column indexes when reading the segment // sort of like the dump segment tool, but build key column indexes when reading the segment
@ -143,12 +147,8 @@ public class BroadcastSegmentIndexedTable implements IndexedTable
for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex < selectors.size(); keyColumnSelectorIndex++) { for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex < selectors.size(); keyColumnSelectorIndex++) {
final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex); final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex);
final int columnPosition = rowSignature.indexOf(keyColumnName); final int columnPosition = rowSignature.indexOf(keyColumnName);
final Map<Object, IntList> keyColumnValueIndex = keyColumnsIndex.get(columnPosition); final RowBasedIndexBuilder keyColumnIndexBuilder = indexBuilders.get(columnPosition);
final Object key = selectors.get(keyColumnSelectorIndex).getObject(); keyColumnIndexBuilder.add(selectors.get(keyColumnSelectorIndex).getObject());
if (key != null) {
final IntList array = keyColumnValueIndex.computeIfAbsent(key, k -> new IntArrayList());
array.add(rowNumber);
}
} }
if (rowNumber % 100_000 == 0) { if (rowNumber % 100_000 == 0) {
@ -166,6 +166,11 @@ public class BroadcastSegmentIndexedTable implements IndexedTable
); );
Integer totalRows = sequence.accumulate(0, (accumulated, in) -> accumulated += in); Integer totalRows = sequence.accumulate(0, (accumulated, in) -> accumulated += in);
this.keyColumnsIndexes = indexBuilders.stream()
.map(builder -> builder != null ? builder.build() : null)
.collect(Collectors.toList());
LOG.info("Created BroadcastSegmentIndexedTable with %s rows.", totalRows); LOG.info("Created BroadcastSegmentIndexedTable with %s rows.", totalRows);
} }
@ -196,7 +201,7 @@ public class BroadcastSegmentIndexedTable implements IndexedTable
@Override @Override
public Index columnIndex(int column) public Index columnIndex(int column)
{ {
return RowBasedIndexedTable.getKeyColumnIndex(column, keyColumnsIndex, rowSignature); return RowBasedIndexedTable.getKeyColumnIndex(column, keyColumnsIndexes);
} }
@Override @Override

View File

@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ReferenceCountedObject; import org.apache.druid.segment.ReferenceCountedObject;
import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.data.ReadableOffset; import org.apache.druid.segment.data.ReadableOffset;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -92,10 +93,37 @@ public interface IndexedTable extends ReferenceCountedObject, Closeable
*/ */
interface Index interface Index
{ {
int NOT_FOUND = -1;
/** /**
* Returns the list of row numbers where the column this Reader is based on contains 'key'. * Returns the natural key type for the index.
*/
ValueType keyType();
/**
* Returns whether keys are unique in this index. If this returns true, then {@link #find(Object)} will only ever
* return a zero- or one-element list.
*/
boolean areKeysUnique();
/**
* Returns the list of row numbers corresponding to "key" in this index.
*
* If "key" is some type other than the natural type {@link #keyType()}, it will be converted before checking
* the index.
*/ */
IntList find(Object key); IntList find(Object key);
/**
* Returns the row number corresponding to "key" in this index, or {@link #NOT_FOUND} if the key does not exist
* in the index.
*
* It is only valid to call this method if {@link #keyType()} is {@link ValueType#LONG} and {@link #areKeysUnique()}
* returns true.
*
* @throws UnsupportedOperationException if preconditions are not met
*/
int findUniqueLong(long key);
} }
/** /**

View File

@ -30,6 +30,7 @@ import it.unimi.dsi.fastutil.ints.IntRBTreeSet;
import it.unimi.dsi.fastutil.ints.IntSet; import it.unimi.dsi.fastutil.ints.IntSet;
import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.query.QueryUnsupportedException; import org.apache.druid.query.QueryUnsupportedException;
import org.apache.druid.segment.BaseDoubleColumnValueSelector; import org.apache.druid.segment.BaseDoubleColumnValueSelector;
@ -59,18 +60,19 @@ import java.util.Map;
import java.util.NoSuchElementException; import java.util.NoSuchElementException;
import java.util.function.Function; import java.util.function.Function;
import java.util.function.IntFunction; import java.util.function.IntFunction;
import java.util.function.Supplier;
import java.util.stream.Collectors; import java.util.stream.Collectors;
public class IndexedTableJoinMatcher implements JoinMatcher public class IndexedTableJoinMatcher implements JoinMatcher
{ {
static final int NO_CONDITION_MATCH = -1;
private static final int UNINITIALIZED_CURRENT_ROW = -1; private static final int UNINITIALIZED_CURRENT_ROW = -1;
// Key column type to use when the actual key type is unknown. // Key column type to use when the actual key type is unknown.
static final ValueType DEFAULT_KEY_TYPE = ValueType.STRING; static final ValueType DEFAULT_KEY_TYPE = ValueType.STRING;
private final IndexedTable table; private final IndexedTable table;
private final List<Supplier<IntIterator>> conditionMatchers; private final List<ConditionMatcher> conditionMatchers;
private final boolean singleRowMatching;
private final IntIterator[] currentMatchedRows; private final IntIterator[] currentMatchedRows;
private final ColumnSelectorFactory selectorFactory; private final ColumnSelectorFactory selectorFactory;
@ -103,13 +105,23 @@ public class IndexedTableJoinMatcher implements JoinMatcher
if (condition.isAlwaysTrue()) { if (condition.isAlwaysTrue()) {
this.conditionMatchers = Collections.singletonList(() -> IntIterators.fromTo(0, table.numRows())); this.conditionMatchers = Collections.singletonList(() -> IntIterators.fromTo(0, table.numRows()));
this.singleRowMatching = false;
} else if (condition.isAlwaysFalse()) { } else if (condition.isAlwaysFalse()) {
this.conditionMatchers = Collections.singletonList(() -> IntIterators.EMPTY_ITERATOR); this.conditionMatchers = Collections.singletonList(() -> IntIterators.EMPTY_ITERATOR);
this.singleRowMatching = false;
} else if (condition.getNonEquiConditions().isEmpty()) { } else if (condition.getNonEquiConditions().isEmpty()) {
this.conditionMatchers = condition.getEquiConditions() final List<Pair<IndexedTable.Index, Equality>> indexes =
.stream() condition.getEquiConditions()
.map(eq -> makeConditionMatcher(table, leftSelectorFactory, eq)) .stream()
.collect(Collectors.toCollection(ArrayList::new)); .map(eq -> Pair.of(getIndex(table, eq), eq))
.collect(Collectors.toCollection(ArrayList::new));
this.conditionMatchers =
indexes.stream()
.map(pair -> makeConditionMatcher(pair.lhs, leftSelectorFactory, pair.rhs))
.collect(Collectors.toList());
this.singleRowMatching = indexes.stream().allMatch(pair -> pair.lhs.areKeysUnique());
} else { } else {
throw new IAE( throw new IAE(
"Cannot build hash-join matcher on non-equi-join condition: %s", "Cannot build hash-join matcher on non-equi-join condition: %s",
@ -117,7 +129,13 @@ public class IndexedTableJoinMatcher implements JoinMatcher
); );
} }
this.currentMatchedRows = new IntIterator[conditionMatchers.size()]; if (!singleRowMatching) {
// Only used by "matchCondition", and only in the multi-row-matching case.
this.currentMatchedRows = new IntIterator[conditionMatchers.size()];
} else {
this.currentMatchedRows = null;
}
ColumnSelectorFactory selectorFactory = table.makeColumnSelectorFactory(joinableOffset, descending, closer); ColumnSelectorFactory selectorFactory = table.makeColumnSelectorFactory(joinableOffset, descending, closer);
this.selectorFactory = selectorFactory != null this.selectorFactory = selectorFactory != null
? selectorFactory ? selectorFactory
@ -131,9 +149,8 @@ public class IndexedTableJoinMatcher implements JoinMatcher
} }
private static Supplier<IntIterator> makeConditionMatcher( private static IndexedTable.Index getIndex(
final IndexedTable table, final IndexedTable table,
final ColumnSelectorFactory selectorFactory,
final Equality condition final Equality condition
) )
{ {
@ -143,15 +160,19 @@ public class IndexedTableJoinMatcher implements JoinMatcher
final int keyColumnNumber = table.rowSignature().indexOf(condition.getRightColumn()); final int keyColumnNumber = table.rowSignature().indexOf(condition.getRightColumn());
final ValueType keyType = return table.columnIndex(keyColumnNumber);
table.rowSignature().getColumnType(condition.getRightColumn()).orElse(DEFAULT_KEY_TYPE); }
final IndexedTable.Index index = table.columnIndex(keyColumnNumber);
private static ConditionMatcher makeConditionMatcher(
final IndexedTable.Index index,
final ColumnSelectorFactory selectorFactory,
final Equality condition
)
{
return ColumnProcessors.makeProcessor( return ColumnProcessors.makeProcessor(
condition.getLeftExpr(), condition.getLeftExpr(),
keyType, index.keyType(),
new ConditionMatcherFactory(keyType, index), new ConditionMatcherFactory(index),
selectorFactory selectorFactory
); );
} }
@ -167,22 +188,39 @@ public class IndexedTableJoinMatcher implements JoinMatcher
{ {
reset(); reset();
for (int i = 0; i < conditionMatchers.size(); i++) { if (singleRowMatching) {
final IntIterator rows = conditionMatchers.get(i).get(); if (conditionMatchers.size() == 1) {
if (rows.hasNext()) { currentRow = conditionMatchers.get(0).matchSingleRow();
currentMatchedRows[i] = rows;
} else { } else {
return; currentRow = conditionMatchers.get(0).matchSingleRow();
for (int i = 1; i < conditionMatchers.size(); i++) {
if (currentRow != conditionMatchers.get(i).matchSingleRow()) {
currentRow = UNINITIALIZED_CURRENT_ROW;
break;
}
}
} }
}
if (currentMatchedRows.length == 1) {
currentIterator = currentMatchedRows[0];
} else { } else {
currentIterator = new SortedIntIntersectionIterator(currentMatchedRows); if (conditionMatchers.size() == 1) {
currentIterator = conditionMatchers.get(0).match();
} else {
for (int i = 0; i < conditionMatchers.size(); i++) {
final IntIterator rows = conditionMatchers.get(i).match();
if (rows.hasNext()) {
currentMatchedRows[i] = rows;
} else {
return;
}
}
currentIterator = new SortedIntIntersectionIterator(currentMatchedRows);
}
advanceCurrentRow();
} }
nextMatch(); addCurrentRowToMatchedRows();
} }
@Override @Override
@ -225,7 +263,7 @@ public class IndexedTableJoinMatcher implements JoinMatcher
}; };
matchingRemainder = true; matchingRemainder = true;
nextMatch(); advanceCurrentRow();
} }
@Override @Override
@ -244,10 +282,7 @@ public class IndexedTableJoinMatcher implements JoinMatcher
public void nextMatch() public void nextMatch()
{ {
advanceCurrentRow(); advanceCurrentRow();
addCurrentRowToMatchedRows();
if (!matchingRemainder && matchedRows != null && hasMatch()) {
matchedRows.add(currentRow);
}
} }
@Override @Override
@ -272,11 +307,36 @@ public class IndexedTableJoinMatcher implements JoinMatcher
} }
} }
private void addCurrentRowToMatchedRows()
{
if (!matchingRemainder && matchedRows != null && hasMatch()) {
matchedRows.add(currentRow);
}
}
interface ConditionMatcher
{
/**
* Returns the first row that matches the current cursor position, or {@link #NO_CONDITION_MATCH} if nothing
* matches.
*/
default int matchSingleRow()
{
final IntIterator it = match();
return it.hasNext() ? it.nextInt() : NO_CONDITION_MATCH;
}
/**
* Returns an iterator for the row numbers that match the current cursor position.
*/
IntIterator match();
}
/** /**
* Makes suppliers that returns the list of IndexedTable rows that match the values from selectors. * Makes suppliers that returns the list of IndexedTable rows that match the values from selectors.
*/ */
@VisibleForTesting @VisibleForTesting
static class ConditionMatcherFactory implements ColumnProcessorFactory<Supplier<IntIterator>> static class ConditionMatcherFactory implements ColumnProcessorFactory<ConditionMatcher>
{ {
@VisibleForTesting @VisibleForTesting
static final int CACHE_MAX_SIZE = 1000; static final int CACHE_MAX_SIZE = 1000;
@ -290,9 +350,9 @@ public class IndexedTableJoinMatcher implements JoinMatcher
@SuppressWarnings("MismatchedQueryAndUpdateOfCollection") // updated via computeIfAbsent @SuppressWarnings("MismatchedQueryAndUpdateOfCollection") // updated via computeIfAbsent
private final LruLoadingHashMap<DimensionSelector, Int2IntListMap> dimensionCaches; private final LruLoadingHashMap<DimensionSelector, Int2IntListMap> dimensionCaches;
ConditionMatcherFactory(ValueType keyType, IndexedTable.Index index) ConditionMatcherFactory(IndexedTable.Index index)
{ {
this.keyType = keyType; this.keyType = index.keyType();
this.index = index; this.index = index;
this.dimensionCaches = new LruLoadingHashMap<>( this.dimensionCaches = new LruLoadingHashMap<>(
@ -325,7 +385,7 @@ public class IndexedTableJoinMatcher implements JoinMatcher
} }
@Override @Override
public Supplier<IntIterator> makeDimensionProcessor(DimensionSelector selector, boolean multiValue) public ConditionMatcher makeDimensionProcessor(DimensionSelector selector, boolean multiValue)
{ {
// NOTE: The slow (cardinality unknown) and fast (cardinality known) code paths below only differ in the calls to // NOTE: The slow (cardinality unknown) and fast (cardinality known) code paths below only differ in the calls to
// getRowNumbers() and getAndCacheRowNumbers(), respectively. The majority of the code path is duplicated to avoid // getRowNumbers() and getAndCacheRowNumbers(), respectively. The majority of the code path is duplicated to avoid
@ -374,7 +434,7 @@ public class IndexedTableJoinMatcher implements JoinMatcher
} }
@Override @Override
public Supplier<IntIterator> makeFloatProcessor(BaseFloatColumnValueSelector selector) public ConditionMatcher makeFloatProcessor(BaseFloatColumnValueSelector selector)
{ {
if (NullHandling.replaceWithDefault()) { if (NullHandling.replaceWithDefault()) {
return () -> index.find(selector.getFloat()).iterator(); return () -> index.find(selector.getFloat()).iterator();
@ -384,7 +444,7 @@ public class IndexedTableJoinMatcher implements JoinMatcher
} }
@Override @Override
public Supplier<IntIterator> makeDoubleProcessor(BaseDoubleColumnValueSelector selector) public ConditionMatcher makeDoubleProcessor(BaseDoubleColumnValueSelector selector)
{ {
if (NullHandling.replaceWithDefault()) { if (NullHandling.replaceWithDefault()) {
return () -> index.find(selector.getDouble()).iterator(); return () -> index.find(selector.getDouble()).iterator();
@ -394,19 +454,58 @@ public class IndexedTableJoinMatcher implements JoinMatcher
} }
@Override @Override
public Supplier<IntIterator> makeLongProcessor(BaseLongColumnValueSelector selector) public ConditionMatcher makeLongProcessor(BaseLongColumnValueSelector selector)
{ {
if (NullHandling.replaceWithDefault()) { if (NullHandling.replaceWithDefault()) {
return () -> index.find(selector.getLong()).iterator(); return new ConditionMatcher()
{
@Override
public int matchSingleRow()
{
return index.findUniqueLong(selector.getLong());
}
@Override
public IntIterator match()
{
return index.find(selector.getLong()).iterator();
}
};
} else { } else {
return () -> selector.isNull() ? IntIterators.EMPTY_ITERATOR : index.find(selector.getLong()).iterator(); return new ConditionMatcher()
{
@Override
public int matchSingleRow()
{
return selector.isNull() ? NO_CONDITION_MATCH : index.findUniqueLong(selector.getLong());
}
@Override
public IntIterator match()
{
return selector.isNull() ? IntIterators.EMPTY_ITERATOR : index.find(selector.getLong()).iterator();
}
};
} }
} }
@Override @Override
public Supplier<IntIterator> makeComplexProcessor(BaseObjectColumnValueSelector<?> selector) public ConditionMatcher makeComplexProcessor(BaseObjectColumnValueSelector<?> selector)
{ {
return () -> IntIterators.EMPTY_ITERATOR; return new ConditionMatcher()
{
@Override
public int matchSingleRow()
{
return NO_CONDITION_MATCH;
}
@Override
public IntIterator match()
{
return IntIterators.EMPTY_ITERATOR;
}
};
} }
} }

View File

@ -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.druid.segment.join.table;
import com.google.common.base.Preconditions;
import it.unimi.dsi.fastutil.ints.IntList;
import it.unimi.dsi.fastutil.ints.IntLists;
import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.column.ValueType;
import java.util.Map;
/**
* An {@link IndexedTable.Index} backed by a Map.
*/
public class MapIndex implements IndexedTable.Index
{
private final ValueType keyType;
private final Map<Object, IntList> index;
private final boolean keysUnique;
private final boolean isLong2ObjectMap;
/**
* Creates a new instance based on a particular map.
*
* @param keyType type of keys in "index"
* @param index a map of keys to matching row numbers
* @param keysUnique whether the keys are unique (if true: all IntLists in the index must be exactly 1 element)
*
* @see RowBasedIndexBuilder#build() the main caller
*/
MapIndex(final ValueType keyType, final Map<Object, IntList> index, final boolean keysUnique)
{
this.keyType = Preconditions.checkNotNull(keyType, "keyType");
this.index = Preconditions.checkNotNull(index, "index");
this.keysUnique = keysUnique;
this.isLong2ObjectMap = index instanceof Long2ObjectMap;
}
@Override
public ValueType keyType()
{
return keyType;
}
@Override
public boolean areKeysUnique()
{
return keysUnique;
}
@Override
public IntList find(Object key)
{
final Object convertedKey = DimensionHandlerUtils.convertObjectToType(key, keyType, false);
if (convertedKey != null) {
final IntList found = index.get(convertedKey);
if (found != null) {
return found;
} else {
return IntLists.EMPTY_LIST;
}
} else {
return IntLists.EMPTY_LIST;
}
}
@Override
public int findUniqueLong(long key)
{
if (isLong2ObjectMap && keysUnique) {
final IntList rows = ((Long2ObjectMap<IntList>) (Map) index).get(key);
assert rows == null || rows.size() == 1;
return rows != null ? rows.getInt(0) : NOT_FOUND;
} else {
throw new UnsupportedOperationException();
}
}
}

View File

@ -0,0 +1,158 @@
/*
* 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.druid.segment.join.table;
import com.google.common.primitives.Ints;
import it.unimi.dsi.fastutil.ints.IntArrayList;
import it.unimi.dsi.fastutil.ints.IntList;
import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
import it.unimi.dsi.fastutil.objects.ObjectIterator;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.column.ValueType;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
/**
* Utility class for creating {@link IndexedTable.Index} instances.
*
* Its main role is to decide which kind of implementation to use.
*/
public class RowBasedIndexBuilder
{
// Long2ObjectOpenHashMap<IntList> is (very) roughly 15x bigger than int[] per entry.
private static final long INT_ARRAY_SPACE_SAVINGS_FACTOR = 15;
// A number that is small enough that we shouldn't worry about making a full array for it. (Yields a 1MB array.)
private static final long INT_ARRAY_SMALL_SIZE_OK = 250_000;
private int currentRow = 0;
private int nullKeys = 0;
private final ValueType keyType;
private final Map<Object, IntList> index;
private long minLongKey = Long.MAX_VALUE;
private long maxLongKey = Long.MIN_VALUE;
public RowBasedIndexBuilder(ValueType keyType)
{
this.keyType = keyType;
if (keyType == ValueType.LONG) {
// We're specializing the type even though we don't specialize usage in this class, for two reasons:
// (1) It's still useful to reduce overall memory footprint.
// (2) MapIndex specifically checks for Long2ObjectMap instances and *does* specialize usage.
final Long2ObjectOpenHashMap<IntList> theMap = new Long2ObjectOpenHashMap<>();
index = (Map) theMap;
} else {
index = new HashMap<>();
}
}
/**
* Add a key to the index. This must be called exactly once per row, even for null values or values that are the
* wrong type, because the builder keeps an internal row-number counter. The builder will handle both nulls and
* mismatched types, so callers do not need to worry about this.
*/
public RowBasedIndexBuilder add(@Nullable final Object key)
{
final Object castKey = DimensionHandlerUtils.convertObjectToType(key, keyType);
if (castKey != null) {
final IntList rowNums = index.computeIfAbsent(castKey, k -> new IntArrayList());
rowNums.add(currentRow);
// Track min, max long value so we can decide later on if it's appropriate to use an array-backed implementation.
if (keyType == ValueType.LONG && (long) castKey < minLongKey) {
minLongKey = (long) castKey;
}
if (keyType == ValueType.LONG && (long) castKey > maxLongKey) {
maxLongKey = (long) castKey;
}
} else {
nullKeys++;
}
currentRow++;
return this;
}
/**
* Create the index. After calling this, the state of the builder is undefined, and you should discard it.
*/
public IndexedTable.Index build()
{
final boolean keysUnique = index.size() == currentRow - nullKeys;
if (keyType == ValueType.LONG && keysUnique && index.size() > 0) {
// May be a good candidate for UniqueLongArrayIndex. Check the range of values as compared to min and max.
long range;
try {
// Add 1 so "range" would be equal to the size of the necessary array.
range = Math.addExact(Math.subtractExact(maxLongKey, minLongKey), 1);
}
catch (ArithmeticException e) {
// Overflow; way too big.
range = 0;
}
// Use a UniqueLongArrayIndex if the range of values is small enough.
final long rangeThreshold = Math.max(
INT_ARRAY_SMALL_SIZE_OK,
Math.min(Integer.MAX_VALUE, INT_ARRAY_SPACE_SAVINGS_FACTOR * index.size())
);
if (range > 0 && range < rangeThreshold) {
final int[] indexAsArray = new int[Ints.checkedCast(range)];
Arrays.fill(indexAsArray, IndexedTable.Index.NOT_FOUND);
// Safe to cast to Long2ObjectMap because the constructor always uses one for long-typed keys.
final ObjectIterator<Long2ObjectMap.Entry<IntList>> entries =
((Long2ObjectMap<IntList>) ((Map) index)).long2ObjectEntrySet().iterator();
while (entries.hasNext()) {
final Long2ObjectMap.Entry<IntList> entry = entries.next();
final IntList rowNums = entry.getValue();
if (rowNums.size() != 1) {
throw new ISE("Expected single element");
}
indexAsArray[Ints.checkedCast(entry.getLongKey() - minLongKey)] = rowNums.getInt(0);
entries.remove();
}
assert index.isEmpty();
// Early return of specialized implementation.
return new UniqueLongArrayIndex(indexAsArray, minLongKey);
}
}
return new MapIndex(keyType, index, keysUnique);
}
}

View File

@ -20,21 +20,15 @@
package org.apache.druid.segment.join.table; package org.apache.druid.segment.join.table;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import it.unimi.dsi.fastutil.ints.IntArrayList;
import it.unimi.dsi.fastutil.ints.IntList;
import it.unimi.dsi.fastutil.ints.IntLists;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.RowAdapter; import org.apache.druid.segment.RowAdapter;
import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.column.ValueType;
import java.io.Closeable; import java.io.Closeable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.Set; import java.util.Set;
import java.util.function.Function; import java.util.function.Function;
@ -47,7 +41,7 @@ import java.util.stream.Collectors;
public class RowBasedIndexedTable<RowType> implements IndexedTable public class RowBasedIndexedTable<RowType> implements IndexedTable
{ {
private final List<RowType> table; private final List<RowType> table;
private final List<Map<Object, IntList>> index; private final List<Index> indexes;
private final RowSignature rowSignature; private final RowSignature rowSignature;
private final List<Function<RowType, Object>> columnFunctions; private final List<Function<RowType, Object>> columnFunctions;
private final Set<String> keyColumns; private final Set<String> keyColumns;
@ -76,33 +70,29 @@ public class RowBasedIndexedTable<RowType> implements IndexedTable
); );
} }
index = new ArrayList<>(rowSignature.size()); indexes = new ArrayList<>(rowSignature.size());
for (int i = 0; i < rowSignature.size(); i++) { for (int i = 0; i < rowSignature.size(); i++) {
final String column = rowSignature.getColumnName(i); final String column = rowSignature.getColumnName(i);
final Map<Object, IntList> m; final Index m;
if (keyColumns.contains(column)) { if (keyColumns.contains(column)) {
final ValueType keyType = final ValueType keyType =
rowSignature.getColumnType(column).orElse(IndexedTableJoinMatcher.DEFAULT_KEY_TYPE); rowSignature.getColumnType(column).orElse(IndexedTableJoinMatcher.DEFAULT_KEY_TYPE);
final RowBasedIndexBuilder builder = new RowBasedIndexBuilder(keyType);
final Function<RowType, Object> columnFunction = columnFunctions.get(i); final Function<RowType, Object> columnFunction = columnFunctions.get(i);
m = new HashMap<>(); for (final RowType row : table) {
builder.add(columnFunction.apply(row));
for (int j = 0; j < table.size(); j++) {
final RowType row = table.get(j);
final Object key = DimensionHandlerUtils.convertObjectToType(columnFunction.apply(row), keyType);
if (key != null) {
final IntList array = m.computeIfAbsent(key, k -> new IntArrayList());
array.add(j);
}
} }
m = builder.build();
} else { } else {
m = null; m = null;
} }
index.add(m); indexes.add(m);
} }
} }
@ -127,7 +117,7 @@ public class RowBasedIndexedTable<RowType> implements IndexedTable
@Override @Override
public Index columnIndex(int column) public Index columnIndex(int column)
{ {
return getKeyColumnIndex(column, index, rowSignature); return getKeyColumnIndex(column, indexes);
} }
@Override @Override
@ -161,30 +151,14 @@ public class RowBasedIndexedTable<RowType> implements IndexedTable
// nothing to close // nothing to close
} }
static Index getKeyColumnIndex(int column, List<Map<Object, IntList>> keyColumnsIndex, RowSignature rowSignature) static Index getKeyColumnIndex(int column, List<Index> indexes)
{ {
final Map<Object, IntList> indexMap = keyColumnsIndex.get(column); final Index index = indexes.get(column);
if (indexMap == null) { if (index == null) {
throw new IAE("Column[%d] is not a key column", column); throw new IAE("Column[%d] is not a key column", column);
} }
final ValueType columnType = return index;
rowSignature.getColumnType(column).orElse(IndexedTableJoinMatcher.DEFAULT_KEY_TYPE);
return key -> {
final Object convertedKey = DimensionHandlerUtils.convertObjectToType(key, columnType, false);
if (convertedKey != null) {
final IntList found = indexMap.get(convertedKey);
if (found != null) {
return found;
} else {
return IntLists.EMPTY_LIST;
}
} else {
return IntLists.EMPTY_LIST;
}
};
} }
} }

View File

@ -0,0 +1,87 @@
/*
* 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.druid.segment.join.table;
import it.unimi.dsi.fastutil.ints.IntList;
import it.unimi.dsi.fastutil.ints.IntLists;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.column.ValueType;
/**
* An {@link IndexedTable.Index} backed by an int array.
*
* This is for long-typed keys whose values all fall in a "reasonable" range.
*/
public class UniqueLongArrayIndex implements IndexedTable.Index
{
private final int[] index;
private final long minKey;
/**
* Create a new instance backed by a given array.
*
* @param index an int array where position {@code i} corresponds to the key {@code i + minKey}
* @param minKey lowest-valued key
*
* @see RowBasedIndexBuilder#build() the main caller
*/
UniqueLongArrayIndex(int[] index, long minKey)
{
this.index = index;
this.minKey = minKey;
}
@Override
public ValueType keyType()
{
return ValueType.LONG;
}
@Override
public boolean areKeysUnique()
{
return true;
}
@Override
public IntList find(Object key)
{
final Long longKey = DimensionHandlerUtils.convertObjectToLong(key);
if (longKey != null) {
final int row = findUniqueLong(longKey);
if (row >= 0) {
return IntLists.singleton(row);
}
}
return IntLists.EMPTY_LIST;
}
@Override
public int findUniqueLong(long key)
{
if (key >= minKey && key < (minKey + index.length)) {
return index[(int) (key - minKey)];
} else {
return NOT_FOUND;
}
}
}

View File

@ -19,9 +19,10 @@
package org.apache.druid.segment.join.table; package org.apache.druid.segment.join.table;
import com.google.common.collect.ImmutableList;
import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntArrayList;
import it.unimi.dsi.fastutil.ints.IntIterator;
import it.unimi.dsi.fastutil.ints.IntList; import it.unimi.dsi.fastutil.ints.IntList;
import it.unimi.dsi.fastutil.ints.IntLists;
import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.QueryUnsupportedException; import org.apache.druid.query.QueryUnsupportedException;
import org.apache.druid.segment.ConstantDimensionSelector; import org.apache.druid.segment.ConstantDimensionSelector;
@ -31,18 +32,18 @@ import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.data.ArrayBasedIndexedInts; import org.apache.druid.segment.data.ArrayBasedIndexedInts;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.runners.Enclosed; import org.junit.experimental.runners.Enclosed;
import org.junit.rules.ExpectedException;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.mockito.Mock; import org.mockito.Mock;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.MockitoAnnotations; import org.mockito.MockitoAnnotations;
import java.util.Collections;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function; import java.util.function.Function;
import java.util.function.IntFunction; import java.util.function.IntFunction;
import java.util.function.Supplier;
@RunWith(Enclosed.class) @RunWith(Enclosed.class)
public class IndexedTableJoinMatcherTest public class IndexedTableJoinMatcherTest
@ -54,6 +55,9 @@ public class IndexedTableJoinMatcherTest
{ {
public static class MakeDimensionProcessorTest public static class MakeDimensionProcessorTest
{ {
@Rule
public ExpectedException expectedException = ExpectedException.none();
@Mock @Mock
private DimensionSelector dimensionSelector; private DimensionSelector dimensionSelector;
@ -63,8 +67,7 @@ public class IndexedTableJoinMatcherTest
NullHandling.initializeForTests(); NullHandling.initializeForTests();
} }
@SuppressWarnings("ReturnValueIgnored") @Test
@Test(expected = QueryUnsupportedException.class)
public void testMatchMultiValuedRowCardinalityUnknownShouldThrowException() public void testMatchMultiValuedRowCardinalityUnknownShouldThrowException()
{ {
MockitoAnnotations.initMocks(this); MockitoAnnotations.initMocks(this);
@ -73,17 +76,18 @@ public class IndexedTableJoinMatcherTest
Mockito.doReturn(DimensionDictionarySelector.CARDINALITY_UNKNOWN).when(dimensionSelector).getValueCardinality(); Mockito.doReturn(DimensionDictionarySelector.CARDINALITY_UNKNOWN).when(dimensionSelector).getValueCardinality();
IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory = IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory =
new IndexedTableJoinMatcher.ConditionMatcherFactory( new IndexedTableJoinMatcher.ConditionMatcherFactory(stringToLengthIndex());
ValueType.STRING, IndexedTableJoinMatcher.ConditionMatcher dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(
IndexedTableJoinMatcherTest::createSingletonIntList dimensionSelector,
); false
Supplier<IntIterator> dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false); );
// Test match should throw exception // Test match should throw exception
dimensionProcessor.get(); expectedException.expect(QueryUnsupportedException.class);
dimensionProcessor.match();
} }
@SuppressWarnings("ReturnValueIgnored") @Test
@Test(expected = QueryUnsupportedException.class)
public void testMatchMultiValuedRowCardinalityKnownShouldThrowException() public void testMatchMultiValuedRowCardinalityKnownShouldThrowException()
{ {
MockitoAnnotations.initMocks(this); MockitoAnnotations.initMocks(this);
@ -92,13 +96,15 @@ public class IndexedTableJoinMatcherTest
Mockito.doReturn(3).when(dimensionSelector).getValueCardinality(); Mockito.doReturn(3).when(dimensionSelector).getValueCardinality();
IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory = IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory =
new IndexedTableJoinMatcher.ConditionMatcherFactory( new IndexedTableJoinMatcher.ConditionMatcherFactory(stringToLengthIndex());
ValueType.STRING, IndexedTableJoinMatcher.ConditionMatcher dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(
IndexedTableJoinMatcherTest::createSingletonIntList dimensionSelector,
); false
Supplier<IntIterator> dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false); );
// Test match should throw exception // Test match should throw exception
dimensionProcessor.get(); expectedException.expect(QueryUnsupportedException.class);
dimensionProcessor.match();
} }
@Test @Test
@ -110,13 +116,16 @@ public class IndexedTableJoinMatcherTest
Mockito.doReturn(DimensionDictionarySelector.CARDINALITY_UNKNOWN).when(dimensionSelector).getValueCardinality(); Mockito.doReturn(DimensionDictionarySelector.CARDINALITY_UNKNOWN).when(dimensionSelector).getValueCardinality();
IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory = IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory =
new IndexedTableJoinMatcher.ConditionMatcherFactory( new IndexedTableJoinMatcher.ConditionMatcherFactory(stringToLengthIndex());
ValueType.STRING, IndexedTableJoinMatcher.ConditionMatcher dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(
IndexedTableJoinMatcherTest::createSingletonIntList dimensionSelector,
); false
Supplier<IntIterator> dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false); );
Assert.assertNotNull(dimensionProcessor.get());
Assert.assertFalse(dimensionProcessor.get().hasNext()); Assert.assertNotNull(dimensionProcessor.match());
Assert.assertFalse(dimensionProcessor.match().hasNext());
Assert.assertEquals(IndexedTableJoinMatcher.NO_CONDITION_MATCH, dimensionProcessor.matchSingleRow());
} }
@Test @Test
@ -128,46 +137,56 @@ public class IndexedTableJoinMatcherTest
Mockito.doReturn(0).when(dimensionSelector).getValueCardinality(); Mockito.doReturn(0).when(dimensionSelector).getValueCardinality();
IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory = IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory =
new IndexedTableJoinMatcher.ConditionMatcherFactory( new IndexedTableJoinMatcher.ConditionMatcherFactory(stringToLengthIndex());
ValueType.STRING, IndexedTableJoinMatcher.ConditionMatcher dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(
IndexedTableJoinMatcherTest::createSingletonIntList dimensionSelector,
); false
Supplier<IntIterator> dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false); );
Assert.assertNotNull(dimensionProcessor.get());
Assert.assertFalse(dimensionProcessor.get().hasNext()); Assert.assertNotNull(dimensionProcessor.match());
Assert.assertFalse(dimensionProcessor.match().hasNext());
Assert.assertEquals(IndexedTableJoinMatcher.NO_CONDITION_MATCH, dimensionProcessor.matchSingleRow());
} }
@Test @Test
public void getsCorrectResultWhenSelectorCardinalityUnknown() public void getsCorrectResultWhenSelectorCardinalityUnknown()
{ {
Supplier<IntIterator> target = makeDimensionProcessor(DimensionDictionarySelector.CARDINALITY_UNKNOWN); IndexedTableJoinMatcher.ConditionMatcher target =
Assert.assertEquals(KEY.length(), target.get().nextInt()); makeConditionMatcher(DimensionDictionarySelector.CARDINALITY_UNKNOWN);
Assert.assertEquals(ImmutableList.of(KEY.length()), new IntArrayList(target.match()));
Assert.assertEquals(KEY.length(), target.matchSingleRow());
} }
@Test @Test
public void getsCorrectResultWhenSelectorCardinalityLow() public void getsCorrectResultWhenSelectorCardinalityLow()
{ {
int lowCardinality = IndexedTableJoinMatcher.ConditionMatcherFactory.CACHE_MAX_SIZE / 10; int lowCardinality = IndexedTableJoinMatcher.ConditionMatcherFactory.CACHE_MAX_SIZE / 10;
Supplier<IntIterator> target = makeDimensionProcessor(lowCardinality); IndexedTableJoinMatcher.ConditionMatcher target = makeConditionMatcher(lowCardinality);
Assert.assertEquals(KEY.length(), target.get().nextInt());
Assert.assertEquals(ImmutableList.of(KEY.length()), new IntArrayList(target.match()));
Assert.assertEquals(KEY.length(), target.matchSingleRow());
} }
@Test @Test
public void getsCorrectResultWhenSelectorCardinalityHigh() public void getsCorrectResultWhenSelectorCardinalityHigh()
{ {
int highCardinality = IndexedTableJoinMatcher.ConditionMatcherFactory.CACHE_MAX_SIZE / 10; int highCardinality = IndexedTableJoinMatcher.ConditionMatcherFactory.CACHE_MAX_SIZE / 10;
Supplier<IntIterator> target = makeDimensionProcessor(highCardinality); IndexedTableJoinMatcher.ConditionMatcher target = makeConditionMatcher(highCardinality);
Assert.assertEquals(KEY.length(), target.get().nextInt());
Assert.assertEquals(ImmutableList.of(KEY.length()), new IntArrayList(target.match()));
Assert.assertEquals(KEY.length(), target.matchSingleRow());
} }
private static Supplier<IntIterator> makeDimensionProcessor(int valueCardinality) private static IndexedTableJoinMatcher.ConditionMatcher makeConditionMatcher(int valueCardinality)
{ {
IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory = IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory =
new IndexedTableJoinMatcher.ConditionMatcherFactory( new IndexedTableJoinMatcher.ConditionMatcherFactory(stringToLengthIndex());
ValueType.STRING, return conditionMatcherFactory.makeDimensionProcessor(
IndexedTableJoinMatcherTest::createSingletonIntList new TestDimensionSelector(KEY, valueCardinality),
); false
return conditionMatcherFactory.makeDimensionProcessor(new TestDimensionSelector(KEY, valueCardinality), false); );
} }
private static class TestDimensionSelector extends ConstantDimensionSelector private static class TestDimensionSelector extends ConstantDimensionSelector
@ -255,7 +274,7 @@ public class IndexedTableJoinMatcherTest
counter = new AtomicLong(0); counter = new AtomicLong(0);
IntFunction<IntList> loader = key -> { IntFunction<IntList> loader = key -> {
counter.incrementAndGet(); counter.incrementAndGet();
return createSingletonIntList(key); return IntLists.singleton(key);
}; };
target = new IndexedTableJoinMatcher.Int2IntListLookupTable(SIZE, loader); target = new IndexedTableJoinMatcher.Int2IntListLookupTable(SIZE, loader);
@ -265,7 +284,7 @@ public class IndexedTableJoinMatcherTest
public void loadsValueIfAbsent() public void loadsValueIfAbsent()
{ {
int key = 1; int key = 1;
Assert.assertEquals(createSingletonIntList(key), target.getAndLoadIfAbsent(key)); Assert.assertEquals(IntLists.singleton(key), target.getAndLoadIfAbsent(key));
Assert.assertEquals(1L, counter.longValue()); Assert.assertEquals(1L, counter.longValue());
} }
@ -273,8 +292,8 @@ public class IndexedTableJoinMatcherTest
public void doesNotLoadIfPresent() public void doesNotLoadIfPresent()
{ {
int key = 1; int key = 1;
Assert.assertEquals(createSingletonIntList(key), target.getAndLoadIfAbsent(key)); Assert.assertEquals(IntLists.singleton(key), target.getAndLoadIfAbsent(key));
Assert.assertEquals(createSingletonIntList(key), target.getAndLoadIfAbsent(key)); Assert.assertEquals(IntLists.singleton(key), target.getAndLoadIfAbsent(key));
Assert.assertEquals(1L, counter.longValue()); Assert.assertEquals(1L, counter.longValue());
} }
} }
@ -291,7 +310,7 @@ public class IndexedTableJoinMatcherTest
counter = new AtomicLong(0); counter = new AtomicLong(0);
IntFunction<IntList> loader = key -> { IntFunction<IntList> loader = key -> {
counter.incrementAndGet(); counter.incrementAndGet();
return createSingletonIntList(key); return IntLists.singleton(key);
}; };
target = new IndexedTableJoinMatcher.Int2IntListLruCache(SIZE, loader); target = new IndexedTableJoinMatcher.Int2IntListLruCache(SIZE, loader);
@ -301,7 +320,7 @@ public class IndexedTableJoinMatcherTest
public void loadsValueIfAbsent() public void loadsValueIfAbsent()
{ {
int key = 1; int key = 1;
Assert.assertEquals(createSingletonIntList(key), target.getAndLoadIfAbsent(key)); Assert.assertEquals(IntLists.singleton(key), target.getAndLoadIfAbsent(key));
Assert.assertEquals(1L, counter.longValue()); Assert.assertEquals(1L, counter.longValue());
} }
@ -309,8 +328,8 @@ public class IndexedTableJoinMatcherTest
public void doesNotLoadIfPresent() public void doesNotLoadIfPresent()
{ {
int key = 1; int key = 1;
Assert.assertEquals(createSingletonIntList(key), target.getAndLoadIfAbsent(key)); Assert.assertEquals(IntLists.singleton(key), target.getAndLoadIfAbsent(key));
Assert.assertEquals(createSingletonIntList(key), target.getAndLoadIfAbsent(key)); Assert.assertEquals(IntLists.singleton(key), target.getAndLoadIfAbsent(key));
Assert.assertEquals(1L, counter.longValue()); Assert.assertEquals(1L, counter.longValue());
} }
@ -321,23 +340,43 @@ public class IndexedTableJoinMatcherTest
int next = start + SIZE; int next = start + SIZE;
for (int key = start; key < next; key++) { for (int key = start; key < next; key++) {
Assert.assertEquals(createSingletonIntList(key), target.getAndLoadIfAbsent(key)); Assert.assertEquals(IntLists.singleton(key), target.getAndLoadIfAbsent(key));
} }
Assert.assertEquals(createSingletonIntList(next), target.getAndLoadIfAbsent(next)); Assert.assertEquals(IntLists.singleton(next), target.getAndLoadIfAbsent(next));
Assert.assertNull(target.get(start)); Assert.assertNull(target.get(start));
Assert.assertEquals(SIZE + 1, counter.longValue()); Assert.assertEquals(SIZE + 1, counter.longValue());
} }
} }
private static IntList createSingletonIntList(Object value) private static IndexedTable.Index stringToLengthIndex()
{ {
return createSingletonIntList(((String) value).length()); return new IndexedTable.Index()
} {
@Override
public ValueType keyType()
{
return ValueType.STRING;
}
private static IntList createSingletonIntList(int value) @Override
{ public boolean areKeysUnique()
return new IntArrayList(Collections.singleton(value)); {
return false;
}
@Override
public IntList find(Object key)
{
return IntLists.singleton(((String) key).length());
}
@Override
public int findUniqueLong(long key)
{
throw new UnsupportedOperationException();
}
};
} }
} }

View File

@ -0,0 +1,182 @@
/*
* 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.druid.segment.join.table;
import it.unimi.dsi.fastutil.ints.IntArrayList;
import it.unimi.dsi.fastutil.ints.IntList;
import org.apache.druid.segment.column.ValueType;
import org.hamcrest.CoreMatchers;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
public class RowBasedIndexBuilderTest
{
@Rule
public ExpectedException expectedException = ExpectedException.none();
@Test
public void test_stringKey_uniqueKeys()
{
final RowBasedIndexBuilder builder =
new RowBasedIndexBuilder(ValueType.STRING)
.add("abc")
.add("")
.add(null)
.add("1")
.add("def");
final IndexedTable.Index index = builder.build();
Assert.assertThat(index, CoreMatchers.instanceOf(MapIndex.class));
Assert.assertEquals(ValueType.STRING, index.keyType());
Assert.assertTrue(index.areKeysUnique());
Assert.assertEquals(intList(0), index.find("abc"));
Assert.assertEquals(intList(1), index.find(""));
Assert.assertEquals(intList(3), index.find(1L));
Assert.assertEquals(intList(3), index.find("1"));
Assert.assertEquals(intList(4), index.find("def"));
Assert.assertEquals(intList(), index.find(null));
Assert.assertEquals(intList(), index.find("nonexistent"));
expectedException.expect(UnsupportedOperationException.class);
index.findUniqueLong(0L);
}
@Test
public void test_stringKey_duplicateKeys()
{
final RowBasedIndexBuilder builder =
new RowBasedIndexBuilder(ValueType.STRING)
.add("abc")
.add("")
.add(null)
.add("abc")
.add("1")
.add("def");
final IndexedTable.Index index = builder.build();
Assert.assertThat(index, CoreMatchers.instanceOf(MapIndex.class));
Assert.assertEquals(ValueType.STRING, index.keyType());
Assert.assertFalse(index.areKeysUnique());
Assert.assertEquals(intList(0, 3), index.find("abc"));
Assert.assertEquals(intList(1), index.find(""));
Assert.assertEquals(intList(4), index.find(1L));
Assert.assertEquals(intList(4), index.find("1"));
Assert.assertEquals(intList(5), index.find("def"));
Assert.assertEquals(intList(), index.find(null));
Assert.assertEquals(intList(), index.find("nonexistent"));
expectedException.expect(UnsupportedOperationException.class);
index.findUniqueLong(0L);
}
@Test
public void test_longKey_uniqueKeys()
{
final RowBasedIndexBuilder builder =
new RowBasedIndexBuilder(ValueType.LONG)
.add(1)
.add(5)
.add(2);
final IndexedTable.Index index = builder.build();
Assert.assertThat(index, CoreMatchers.instanceOf(UniqueLongArrayIndex.class));
Assert.assertEquals(ValueType.LONG, index.keyType());
Assert.assertTrue(index.areKeysUnique());
Assert.assertEquals(intList(0), index.find(1L));
Assert.assertEquals(intList(1), index.find(5L));
Assert.assertEquals(intList(2), index.find(2L));
Assert.assertEquals(intList(), index.find(3L));
Assert.assertEquals(0, index.findUniqueLong(1L));
Assert.assertEquals(1, index.findUniqueLong(5L));
Assert.assertEquals(2, index.findUniqueLong(2L));
Assert.assertEquals(IndexedTable.Index.NOT_FOUND, index.findUniqueLong(3L));
}
@Test
public void test_longKey_uniqueKeys_farApart()
{
final RowBasedIndexBuilder builder =
new RowBasedIndexBuilder(ValueType.LONG)
.add(1)
.add(10_000_000)
.add(2);
final IndexedTable.Index index = builder.build();
Assert.assertThat(index, CoreMatchers.instanceOf(MapIndex.class));
Assert.assertEquals(ValueType.LONG, index.keyType());
Assert.assertTrue(index.areKeysUnique());
Assert.assertEquals(intList(0), index.find(1L));
Assert.assertEquals(intList(1), index.find(10_000_000L));
Assert.assertEquals(intList(2), index.find(2L));
Assert.assertEquals(intList(), index.find(3L));
Assert.assertEquals(0, index.findUniqueLong(1L));
Assert.assertEquals(1, index.findUniqueLong(10_000_000L));
Assert.assertEquals(2, index.findUniqueLong(2L));
Assert.assertEquals(IndexedTable.Index.NOT_FOUND, index.findUniqueLong(3L));
}
@Test
public void test_longKey_duplicateKeys()
{
final RowBasedIndexBuilder builder =
new RowBasedIndexBuilder(ValueType.LONG)
.add(1)
.add(5)
.add(1)
.add(2);
final IndexedTable.Index index = builder.build();
Assert.assertThat(index, CoreMatchers.instanceOf(MapIndex.class));
Assert.assertEquals(ValueType.LONG, index.keyType());
Assert.assertFalse(index.areKeysUnique());
Assert.assertEquals(intList(0, 2), index.find("1"));
Assert.assertEquals(intList(0, 2), index.find(1));
Assert.assertEquals(intList(0, 2), index.find(1L));
Assert.assertEquals(intList(1), index.find(5L));
Assert.assertEquals(intList(3), index.find(2L));
Assert.assertEquals(intList(), index.find(3L));
expectedException.expect(UnsupportedOperationException.class);
index.findUniqueLong(5L);
}
public IntList intList(final int... ints)
{
final IntArrayList retVal = new IntArrayList(ints.length);
for (int i : ints) {
retVal.add(i);
}
return retVal;
}
}