diff --git a/core/src/main/java/org/apache/druid/common/config/NullHandling.java b/core/src/main/java/org/apache/druid/common/config/NullHandling.java index de381554ee4a..bd0f0ee8ae51 100644 --- a/core/src/main/java/org/apache/druid/common/config/NullHandling.java +++ b/core/src/main/java/org/apache/druid/common/config/NullHandling.java @@ -118,6 +118,29 @@ public static Double defaultDoubleValue() return replaceWithDefault() ? ZERO_DOUBLE : null; } + /** + * Returns the default value for an object of the provided class. Will be null in SQL-compatible null handling mode. + * May be null or some non-null default value when not in SQL-compatible null handling mode. + */ + @Nullable + @SuppressWarnings("unchecked") + public static T defaultValueForClass(final Class clazz) + { + if (clazz == Float.class) { + return (T) defaultFloatValue(); + } else if (clazz == Double.class) { + return (T) defaultDoubleValue(); + } else if (clazz == Long.class) { + return (T) defaultLongValue(); + } else if (clazz == Number.class) { + return (T) defaultDoubleValue(); + } else if (clazz == String.class) { + return (T) defaultStringValue(); + } else { + return null; + } + } + public static boolean isNullOrEquivalent(@Nullable String value) { return replaceWithDefault() ? Strings.isNullOrEmpty(value) : value == null; diff --git a/core/src/main/java/org/apache/druid/math/expr/Exprs.java b/core/src/main/java/org/apache/druid/math/expr/Exprs.java new file mode 100644 index 000000000000..b1a34386eb6e --- /dev/null +++ b/core/src/main/java/org/apache/druid/math/expr/Exprs.java @@ -0,0 +1,71 @@ +/* + * 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.math.expr; + +import org.apache.druid.java.util.common.Pair; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Stack; + +public class Exprs +{ + /** + * Decomposes any expr into a list of exprs that, if ANDed together, are equivalent to the input expr. + * + * @param expr any expr + * + * @return list of exprs that, if ANDed together, are equivalent to the input expr + */ + public static List decomposeAnd(final Expr expr) + { + final List retVal = new ArrayList<>(); + final Stack stack = new Stack<>(); + stack.push(expr); + + while (!stack.empty()) { + final Expr current = stack.pop(); + + if (current instanceof BinAndExpr) { + stack.push(((BinAndExpr) current).right); + stack.push(((BinAndExpr) current).left); + } else { + retVal.add(current); + } + } + + return retVal; + } + + /** + * Decomposes an equality expr into the left- and right-hand side. + * + * @return decomposed equality, or empty if the input expr was not an equality expr + */ + public static Optional> decomposeEquals(final Expr expr) + { + if (expr instanceof BinEqExpr) { + return Optional.of(Pair.of(((BinEqExpr) expr).left, ((BinEqExpr) expr).right)); + } else { + return Optional.empty(); + } + } +} diff --git a/core/src/test/java/org/apache/druid/common/config/NullHandlingTest.java b/core/src/test/java/org/apache/druid/common/config/NullHandlingTest.java new file mode 100644 index 000000000000..f57b7e5cd923 --- /dev/null +++ b/core/src/test/java/org/apache/druid/common/config/NullHandlingTest.java @@ -0,0 +1,90 @@ +/* + * 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.common.config; + +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +public class NullHandlingTest +{ + @BeforeClass + public static void setUpClass() + { + NullHandling.initializeForTests(); + } + + @Test + public void test_defaultValueForClass_float() + { + Assert.assertEquals( + NullHandling.replaceWithDefault() ? 0f : null, + NullHandling.defaultValueForClass(Float.class) + ); + } + + @Test + public void test_defaultValueForClass_double() + { + Assert.assertEquals( + NullHandling.replaceWithDefault() ? 0d : null, + NullHandling.defaultValueForClass(Double.class) + ); + } + + @Test + public void test_defaultValueForClass_integer() + { + Assert.assertNull(NullHandling.defaultValueForClass(Integer.class)); + } + + @Test + public void test_defaultValueForClass_long() + { + Assert.assertEquals( + NullHandling.replaceWithDefault() ? 0L : null, + NullHandling.defaultValueForClass(Long.class) + ); + } + + @Test + public void test_defaultValueForClass_number() + { + Assert.assertEquals( + NullHandling.replaceWithDefault() ? 0d : null, + NullHandling.defaultValueForClass(Number.class) + ); + } + + @Test + public void test_defaultValueForClass_string() + { + Assert.assertEquals( + NullHandling.replaceWithDefault() ? "" : null, + NullHandling.defaultValueForClass(String.class) + ); + } + + @Test + public void test_defaultValueForClass_object() + { + Assert.assertNull(NullHandling.defaultValueForClass(Object.class)); + } +} diff --git a/core/src/test/java/org/apache/druid/math/expr/ExprsTest.java b/core/src/test/java/org/apache/druid/math/expr/ExprsTest.java new file mode 100644 index 000000000000..a5b6844d5a23 --- /dev/null +++ b/core/src/test/java/org/apache/druid/math/expr/ExprsTest.java @@ -0,0 +1,99 @@ +/* + * 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.math.expr; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.Pair; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +public class ExprsTest +{ + @Test + public void test_decomposeAnd_notAnAnd() + { + final List decomposed = Exprs.decomposeAnd(new IdentifierExpr("foo")); + + // Expr instances don't, in general, implement value-based equals and hashCode. So we need to verify each field. + Assert.assertEquals(1, decomposed.size()); + Assert.assertThat(decomposed.get(0), CoreMatchers.instanceOf(IdentifierExpr.class)); + Assert.assertEquals("foo", ((IdentifierExpr) decomposed.get(0)).getIdentifier()); + } + + @Test + public void test_decomposeAnd_basic() + { + final List decomposed = Exprs.decomposeAnd( + new BinAndExpr( + "&&", + new BinAndExpr("&&", new IdentifierExpr("foo"), new IdentifierExpr("bar")), + new BinAndExpr("&&", new IdentifierExpr("baz"), new IdentifierExpr("qux")) + ) + ); + + // Expr instances don't, in general, implement value-based equals and hashCode. So we need to verify each field. + Assert.assertEquals(4, decomposed.size()); + + for (Expr expr : decomposed) { + Assert.assertThat(expr, CoreMatchers.instanceOf(IdentifierExpr.class)); + } + + final List identifiers = decomposed.stream() + .map(expr -> ((IdentifierExpr) expr).getIdentifier()) + .collect(Collectors.toList()); + + Assert.assertEquals( + ImmutableList.of("foo", "bar", "baz", "qux"), + identifiers + ); + } + + @Test + public void test_decomposeEquals_notAnEquals() + { + final Optional> optionalPair = Exprs.decomposeEquals(new IdentifierExpr("foo")); + Assert.assertFalse(optionalPair.isPresent()); + } + + @Test + public void test_decomposeEquals_basic() + { + final Optional> optionalPair = Exprs.decomposeEquals( + new BinEqExpr( + "==", + new IdentifierExpr("foo"), + new IdentifierExpr("bar") + ) + ); + + Assert.assertTrue(optionalPair.isPresent()); + + final Pair pair = optionalPair.get(); + Assert.assertThat(pair.lhs, CoreMatchers.instanceOf(IdentifierExpr.class)); + Assert.assertThat(pair.rhs, CoreMatchers.instanceOf(IdentifierExpr.class)); + Assert.assertEquals("foo", ((IdentifierExpr) pair.lhs).getIdentifier()); + Assert.assertEquals("bar", ((IdentifierExpr) pair.rhs).getIdentifier()); + } +} diff --git a/extensions-core/lookups-cached-single/src/main/java/org/apache/druid/server/lookup/LoadingLookup.java b/extensions-core/lookups-cached-single/src/main/java/org/apache/druid/server/lookup/LoadingLookup.java index 14045a5564cf..af346e2b45aa 100644 --- a/extensions-core/lookups-cached-single/src/main/java/org/apache/druid/server/lookup/LoadingLookup.java +++ b/extensions-core/lookups-cached-single/src/main/java/org/apache/druid/server/lookup/LoadingLookup.java @@ -29,6 +29,7 @@ import javax.annotation.Nullable; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; @@ -91,7 +92,7 @@ public List unapply(@Nullable final String value) // valueEquivalent is null only for SQL Compatible Null Behavior // otherwise null will be replaced with empty string in nullToEmptyIfNeeded above. // null value maps to empty list when SQL Compatible - return Collections.EMPTY_LIST; + return Collections.emptyList(); } final List retList; try { @@ -100,25 +101,20 @@ public List unapply(@Nullable final String value) } catch (ExecutionException e) { LOGGER.debug("list of keys not found for value [%s]", value); - return Collections.EMPTY_LIST; + return Collections.emptyList(); } } - public synchronized void close() + @Override + public boolean canIterate() { - if (isOpen.getAndSet(false)) { - LOGGER.info("Closing loading cache [%s]", id); - loadingCache.close(); - reverseLoadingCache.close(); - } else { - LOGGER.info("Closing already closed lookup"); - return; - } + return false; } - public boolean isOpen() + @Override + public Iterable> iterable() { - return isOpen.get(); + throw new UnsupportedOperationException("Cannot iterate"); } @Override @@ -145,6 +141,22 @@ public String call() } } + public synchronized void close() + { + if (isOpen.getAndSet(false)) { + LOGGER.info("Closing loading cache [%s]", id); + loadingCache.close(); + reverseLoadingCache.close(); + } else { + LOGGER.info("Closing already closed lookup"); + } + } + + public boolean isOpen() + { + return isOpen.get(); + } + private class UnapplyCallable implements Callable> { private final String value; diff --git a/extensions-core/lookups-cached-single/src/main/java/org/apache/druid/server/lookup/PollingLookup.java b/extensions-core/lookups-cached-single/src/main/java/org/apache/druid/server/lookup/PollingLookup.java index d768c7486ca0..10ccc9b97dc8 100644 --- a/extensions-core/lookups-cached-single/src/main/java/org/apache/druid/server/lookup/PollingLookup.java +++ b/extensions-core/lookups-cached-single/src/main/java/org/apache/druid/server/lookup/PollingLookup.java @@ -35,6 +35,7 @@ import javax.annotation.Nullable; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -165,6 +166,18 @@ public List unapply(@Nullable final String value) } } + @Override + public boolean canIterate() + { + return false; + } + + @Override + public Iterable> iterable() + { + throw new UnsupportedOperationException("Cannot iterate"); + } + @Override public byte[] getCacheKey() { diff --git a/processing/pom.xml b/processing/pom.xml index 777702df0fb8..0629c5837ffc 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -185,6 +185,11 @@ caliper test + + nl.jqno.equalsverifier + equalsverifier + test + pl.pragmatists JUnitParams diff --git a/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java index 737d0ec8ccf3..b1c740f77c4e 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java @@ -32,6 +32,7 @@ import java.util.Objects; /** + * */ public class DefaultDimensionSpec implements DimensionSpec { @@ -136,6 +137,12 @@ public boolean preservesOrdering() return true; } + @Override + public DimensionSpec withDimension(String newDimension) + { + return new DefaultDimensionSpec(newDimension, this.outputName, this.outputType); + } + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java index 9f3eb2ae0317..31ba365ad20d 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java @@ -83,4 +83,10 @@ default boolean canVectorize() } boolean preservesOrdering(); + + /** + * Returns a copy of this DimensionSpec with the underlying dimension (the value of {@link #getDimension()}) + * replaced by "newDimension". + */ + DimensionSpec withDimension(String newDimension); } diff --git a/processing/src/main/java/org/apache/druid/query/dimension/ExtractionDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/ExtractionDimensionSpec.java index 71885064de3f..5f84f8d6ee5b 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/ExtractionDimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/ExtractionDimensionSpec.java @@ -125,6 +125,12 @@ public boolean preservesOrdering() return extractionFn.preservesOrdering(); } + @Override + public DimensionSpec withDimension(String newDimension) + { + return new ExtractionDimensionSpec(newDimension, outputName, outputType, extractionFn); + } + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/dimension/ListFilteredDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/ListFilteredDimensionSpec.java index ef7cdc911320..777483b9e51c 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/ListFilteredDimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/ListFilteredDimensionSpec.java @@ -35,6 +35,7 @@ import java.util.Set; /** + * */ public class ListFilteredDimensionSpec extends BaseFilteredDimensionSpec { @@ -171,6 +172,12 @@ public byte[] getCacheKey() return filterCacheKey.array(); } + @Override + public DimensionSpec withDimension(String newDimension) + { + return new ListFilteredDimensionSpec(delegate.withDimension(newDimension), values, isWhitelist); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/dimension/LookupDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/LookupDimensionSpec.java index d245efb78c32..7e8923653d63 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/LookupDimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/LookupDimensionSpec.java @@ -200,6 +200,21 @@ public boolean preservesOrdering() return getExtractionFn().preservesOrdering(); } + @Override + public DimensionSpec withDimension(String newDimension) + { + return new LookupDimensionSpec( + newDimension, + outputName, + lookup, + retainMissingValue, + replaceMissingValueWith, + name, + optimize, + lookupExtractorFactoryContainerProvider + ); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpec.java index d3f2919b1635..3cf8981044ba 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpec.java @@ -33,6 +33,7 @@ import java.nio.ByteBuffer; /** + * */ public class PrefixFilteredDimensionSpec extends BaseFilteredDimensionSpec { @@ -109,6 +110,12 @@ public byte[] getCacheKey() .array(); } + @Override + public DimensionSpec withDimension(String newDimension) + { + return new PrefixFilteredDimensionSpec(delegate.withDimension(newDimension), prefix); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/dimension/RegexFilteredDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/RegexFilteredDimensionSpec.java index 84883c6d403c..5e4101f7661a 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/RegexFilteredDimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/RegexFilteredDimensionSpec.java @@ -34,6 +34,7 @@ import java.util.regex.Pattern; /** + * */ public class RegexFilteredDimensionSpec extends BaseFilteredDimensionSpec { @@ -113,6 +114,12 @@ public byte[] getCacheKey() .array(); } + @Override + public DimensionSpec withDimension(String newDimension) + { + return new RegexFilteredDimensionSpec(delegate.withDimension(newDimension), pattern); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/extraction/MapLookupExtractor.java b/processing/src/main/java/org/apache/druid/query/extraction/MapLookupExtractor.java index f3320c00f5e5..23096020b60f 100644 --- a/processing/src/main/java/org/apache/druid/query/extraction/MapLookupExtractor.java +++ b/processing/src/main/java/org/apache/druid/query/extraction/MapLookupExtractor.java @@ -122,6 +122,18 @@ public byte[] getCacheKey() } } + @Override + public boolean canIterate() + { + return true; + } + + @Override + public Iterable> iterable() + { + return map.entrySet(); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnStrategizer.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactory.java similarity index 68% rename from processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnStrategizer.java rename to processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactory.java index 7b970e20ce41..5ca511f5a59e 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnStrategizer.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactory.java @@ -19,27 +19,27 @@ package org.apache.druid.query.filter.vector; -import org.apache.druid.query.dimension.VectorColumnStrategizer; +import org.apache.druid.segment.VectorColumnProcessorFactory; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; import org.apache.druid.segment.vector.VectorValueSelector; -public class VectorValueMatcherColumnStrategizer implements VectorColumnStrategizer +public class VectorValueMatcherColumnProcessorFactory implements VectorColumnProcessorFactory { - private static final VectorValueMatcherColumnStrategizer INSTANCE = new VectorValueMatcherColumnStrategizer(); + private static final VectorValueMatcherColumnProcessorFactory INSTANCE = new VectorValueMatcherColumnProcessorFactory(); - private VectorValueMatcherColumnStrategizer() + private VectorValueMatcherColumnProcessorFactory() { // Singleton. } - public static VectorValueMatcherColumnStrategizer instance() + public static VectorValueMatcherColumnProcessorFactory instance() { return INSTANCE; } @Override - public VectorValueMatcherFactory makeSingleValueDimensionStrategy( + public VectorValueMatcherFactory makeSingleValueDimensionProcessor( final SingleValueDimensionVectorSelector selector ) { @@ -47,7 +47,7 @@ public VectorValueMatcherFactory makeSingleValueDimensionStrategy( } @Override - public VectorValueMatcherFactory makeMultiValueDimensionStrategy( + public VectorValueMatcherFactory makeMultiValueDimensionProcessor( final MultiValueDimensionVectorSelector selector ) { @@ -55,19 +55,19 @@ public VectorValueMatcherFactory makeMultiValueDimensionStrategy( } @Override - public VectorValueMatcherFactory makeFloatStrategy(final VectorValueSelector selector) + public VectorValueMatcherFactory makeFloatProcessor(final VectorValueSelector selector) { return new FloatVectorValueMatcher(selector); } @Override - public VectorValueMatcherFactory makeDoubleStrategy(final VectorValueSelector selector) + public VectorValueMatcherFactory makeDoubleProcessor(final VectorValueSelector selector) { return new DoubleVectorValueMatcher(selector); } @Override - public VectorValueMatcherFactory makeLongStrategy(final VectorValueSelector selector) + public VectorValueMatcherFactory makeLongProcessor(final VectorValueSelector selector) { return new LongVectorValueMatcher(selector); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java index 28147fb15792..ed74800ddc17 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java @@ -42,6 +42,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index afb1a1fad780..7c6a0b0d0113 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -53,7 +53,6 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryHelper; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory; import org.apache.druid.query.groupby.epinephelinae.Grouper.BufferComparator; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; @@ -66,6 +65,8 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.RowAdapter; +import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; @@ -347,8 +348,8 @@ public static ColumnSelectorFactory createResultRowBasedColumnSelectorFactory( final Supplier supplier ) { - final RowBasedColumnSelectorFactory.RowAdapter adapter = - new RowBasedColumnSelectorFactory.RowAdapter() + final RowAdapter adapter = + new RowAdapter() { @Override public ToLongFunction timestampFunction() @@ -362,7 +363,7 @@ public ToLongFunction timestampFunction() } @Override - public Function rawFunction(final String columnName) + public Function columnFunction(final String columnName) { final int columnIndex = query.getResultRowPositionLookup().getInt(columnName); if (columnIndex < 0) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnStrategizer.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java similarity index 65% rename from processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnStrategizer.java rename to processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java index c14041cdeae4..b42d34e69cb1 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnStrategizer.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java @@ -19,51 +19,51 @@ package org.apache.druid.query.groupby.epinephelinae.vector; -import org.apache.druid.query.dimension.VectorColumnStrategizer; +import org.apache.druid.segment.VectorColumnProcessorFactory; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; import org.apache.druid.segment.vector.VectorValueSelector; -public class GroupByVectorColumnStrategizer implements VectorColumnStrategizer +public class GroupByVectorColumnProcessorFactory implements VectorColumnProcessorFactory { - private static final GroupByVectorColumnStrategizer INSTANCE = new GroupByVectorColumnStrategizer(); + private static final GroupByVectorColumnProcessorFactory INSTANCE = new GroupByVectorColumnProcessorFactory(); - private GroupByVectorColumnStrategizer() + private GroupByVectorColumnProcessorFactory() { // Singleton. } - public static GroupByVectorColumnStrategizer instance() + public static GroupByVectorColumnProcessorFactory instance() { return INSTANCE; } @Override - public GroupByVectorColumnSelector makeSingleValueDimensionStrategy(final SingleValueDimensionVectorSelector selector) + public GroupByVectorColumnSelector makeSingleValueDimensionProcessor(final SingleValueDimensionVectorSelector selector) { return new SingleValueStringGroupByVectorColumnSelector(selector); } @Override - public GroupByVectorColumnSelector makeMultiValueDimensionStrategy(final MultiValueDimensionVectorSelector selector) + public GroupByVectorColumnSelector makeMultiValueDimensionProcessor(final MultiValueDimensionVectorSelector selector) { throw new UnsupportedOperationException("Multi-value dimensions not yet implemented for vectorized groupBys"); } @Override - public GroupByVectorColumnSelector makeFloatStrategy(final VectorValueSelector selector) + public GroupByVectorColumnSelector makeFloatProcessor(final VectorValueSelector selector) { return new FloatGroupByVectorColumnSelector(selector); } @Override - public GroupByVectorColumnSelector makeDoubleStrategy(final VectorValueSelector selector) + public GroupByVectorColumnSelector makeDoubleProcessor(final VectorValueSelector selector) { return new DoubleGroupByVectorColumnSelector(selector); } @Override - public GroupByVectorColumnSelector makeLongStrategy(final VectorValueSelector selector) + public GroupByVectorColumnSelector makeLongProcessor(final VectorValueSelector selector) { return new LongGroupByVectorColumnSelector(selector); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java index c2e13bd6e8d9..aab0412529d5 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java @@ -149,7 +149,7 @@ public void close() dimensionSpec -> DimensionHandlerUtils.makeVectorProcessor( dimensionSpec, - GroupByVectorColumnStrategizer.instance(), + GroupByVectorColumnProcessorFactory.instance(), columnSelectorFactory ) ).collect(Collectors.toList()); diff --git a/processing/src/main/java/org/apache/druid/query/lookup/LookupExtractor.java b/processing/src/main/java/org/apache/druid/query/lookup/LookupExtractor.java index d99736b303af..f806a5555ab3 100644 --- a/processing/src/main/java/org/apache/druid/query/lookup/LookupExtractor.java +++ b/processing/src/main/java/org/apache/druid/query/lookup/LookupExtractor.java @@ -100,12 +100,23 @@ public Map> unapplyAll(Iterable values) return map; } + /** + * Returns true if this lookup extractor's {@link #iterable()} method will return a valid iterator. + */ + public abstract boolean canIterate(); + + /** + * Returns an Iterable that iterates over the keys and values in this lookup extractor. + * + * @throws UnsupportedOperationException if {@link #canIterate()} returns false. + */ + public abstract Iterable> iterable(); + /** * Create a cache key for use in results caching * * @return A byte array that can be used to uniquely identify if results of a prior lookup can use the cached values */ - public abstract byte[] getCacheKey(); // make this abstract again once @drcrallen fix the metmax lookup implementation. diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 88428c38e169..9215e1102b37 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -49,7 +49,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory; +import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.joda.time.DateTime; import java.util.Collections; diff --git a/processing/src/main/java/org/apache/druid/segment/ColumnProcessorFactory.java b/processing/src/main/java/org/apache/druid/segment/ColumnProcessorFactory.java new file mode 100644 index 000000000000..86600f3473a3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/ColumnProcessorFactory.java @@ -0,0 +1,56 @@ +/* + * 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; + +import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory; +import org.apache.druid.segment.column.ValueType; + +/** + * Class that encapsulates knowledge about how to create "column processors", which are... objects that process columns + * and want to have type-specific logic. Used by {@link ColumnProcessors#makeProcessor}. + * + * Column processors can be any type "T". The idea is that a ColumnProcessorFactory embodies the logic for wrapping + * and processing selectors of various types, and so enables nice code design, where type-dependent code is not + * sprinkled throughout. + * + * @see VectorColumnProcessorFactory the vectorized version + * @see ColumnProcessors#makeProcessor which uses these, and which is responsible for + * determining which type of selector to use for a given column + * @see ColumnSelectorStrategyFactory which serves a similar purpose and may be replaced by this in the future + * @see DimensionHandlerUtils#createColumnSelectorPluses which accepts {@link ColumnSelectorStrategyFactory} and is + * similar to {@link ColumnProcessors#makeProcessor} + */ +public interface ColumnProcessorFactory +{ + /** + * This default type will be used when the underlying column has an unknown type. + */ + ValueType defaultType(); + + T makeDimensionProcessor(DimensionSelector selector); + + T makeFloatProcessor(BaseFloatColumnValueSelector selector); + + T makeDoubleProcessor(BaseDoubleColumnValueSelector selector); + + T makeLongProcessor(BaseLongColumnValueSelector selector); + + T makeComplexProcessor(BaseObjectColumnValueSelector selector); +} diff --git a/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java b/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java new file mode 100644 index 000000000000..676e6e78b5ec --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java @@ -0,0 +1,144 @@ +/* + * 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; + +import com.google.common.base.Function; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.math.expr.Expr; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.virtual.ExpressionSelectors; + +import javax.annotation.Nullable; + +/** + * Creates "column processors", which are objects that wrap a single input column and provide some functionality on + * top of it. + * + * @see DimensionHandlerUtils#createColumnSelectorPlus which this may eventually replace + * @see DimensionHandlerUtils#makeVectorProcessor which creates similar, vectorized processors; may eventually be moved + * into this class. + */ +public class ColumnProcessors +{ + /** + * Make a processor for a particular named column. + * + * @param column the column + * @param processorFactory the processor factory + * @param selectorFactory the column selector factory + * @param processor type + */ + public static T makeProcessor( + final String column, + final ColumnProcessorFactory processorFactory, + final ColumnSelectorFactory selectorFactory + ) + { + return makeProcessorInternal( + factory -> getColumnType(factory, column), + factory -> factory.makeDimensionSelector(DefaultDimensionSpec.of(column)), + factory -> factory.makeColumnValueSelector(column), + processorFactory, + selectorFactory + ); + } + + /** + * Make a processor for a particular expression. If the expression is a simple identifier, this behaves identically + * to {@link #makeProcessor(String, ColumnProcessorFactory, ColumnSelectorFactory)} and accesses the column directly. + * Otherwise, it uses an expression selector of type {@code exprTypeHint}. + * + * @param expr the parsed expression + * @param processorFactory the processor factory + * @param selectorFactory the column selector factory + * @param processor type + */ + public static T makeProcessor( + final Expr expr, + final ValueType exprTypeHint, + final ColumnProcessorFactory processorFactory, + final ColumnSelectorFactory selectorFactory + ) + { + if (expr.getIdentifierIfIdentifier() != null) { + // If expr is an identifier, treat this the same way as a direct column reference. + return makeProcessor(expr.getIdentifierIfIdentifier(), processorFactory, selectorFactory); + } else { + return makeProcessorInternal( + factory -> exprTypeHint, + factory -> ExpressionSelectors.makeDimensionSelector(factory, expr, null), + factory -> ExpressionSelectors.makeColumnValueSelector(factory, expr), + processorFactory, + selectorFactory + ); + } + } + + /** + * Creates "column processors", which are objects that wrap a single input column and provide some + * functionality on top of it. + * + * @param inputTypeFn function that returns the "natural" input type of the column being processed. This is + * permitted to return null; if it does, then processorFactory.defaultType() will be used. + * @param dimensionSelectorFn function that creates a DimensionSelector for the column being processed. Will be + * called if the column type is string. + * @param valueSelectorFunction function that creates a ColumnValueSelector for the column being processed. Will be + * called if the column type is long, float, double, or complex. + * @param processorFactory object that encapsulates the knowledge about how to create processors + * @param selectorFactory column selector factory used for creating the vector processor + * + * @see DimensionHandlerUtils#makeVectorProcessor the vectorized version + */ + private static T makeProcessorInternal( + final Function inputTypeFn, + final Function dimensionSelectorFn, + final Function> valueSelectorFunction, + final ColumnProcessorFactory processorFactory, + final ColumnSelectorFactory selectorFactory + ) + { + final ValueType type = inputTypeFn.apply(selectorFactory); + final ValueType effectiveType = type != null ? type : processorFactory.defaultType(); + + switch (effectiveType) { + case STRING: + return processorFactory.makeDimensionProcessor(dimensionSelectorFn.apply(selectorFactory)); + case LONG: + return processorFactory.makeLongProcessor(valueSelectorFunction.apply(selectorFactory)); + case FLOAT: + return processorFactory.makeFloatProcessor(valueSelectorFunction.apply(selectorFactory)); + case DOUBLE: + return processorFactory.makeDoubleProcessor(valueSelectorFunction.apply(selectorFactory)); + case COMPLEX: + return processorFactory.makeComplexProcessor(valueSelectorFunction.apply(selectorFactory)); + default: + throw new ISE("Unsupported type[%s]", effectiveType); + } + } + + @Nullable + private static ValueType getColumnType(final ColumnSelectorFactory selectorFactory, final String columnName) + { + final ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(columnName); + return capabilities == null ? null : capabilities.getType(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java b/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java index e9396c504372..568d4c8f2de6 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java @@ -34,7 +34,6 @@ import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.query.dimension.VectorColumnStrategizer; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ValueType; @@ -119,6 +118,8 @@ public static List getValueTypesFromDimensionSpecs(List ColumnSelectorPlus createColumnSelectorPlus( ColumnSelectorStrategyFactory strategyFactory, @@ -146,6 +147,8 @@ public static Colum * @param columnSelectorFactory Used to create value selectors for columns. * * @return An array of ColumnSelectorPlus objects, in the order of the columns specified in dimensionSpecs + * + * @see ColumnProcessors#makeProcessor which may replace this in the future */ public static //CHECKSTYLE.OFF: Indentation @@ -249,11 +252,12 @@ private static Colu /** * Equivalent to calling makeVectorProcessor(DefaultDimensionSpec.of(column), strategyFactory, selectorFactory). * - * @see #makeVectorProcessor(DimensionSpec, VectorColumnStrategizer, VectorColumnSelectorFactory) + * @see #makeVectorProcessor(DimensionSpec, VectorColumnProcessorFactory, VectorColumnSelectorFactory) + * @see ColumnProcessors#makeProcessor the non-vectorized version */ public static T makeVectorProcessor( final String column, - final VectorColumnStrategizer strategyFactory, + final VectorColumnProcessorFactory strategyFactory, final VectorColumnSelectorFactory selectorFactory ) { @@ -269,10 +273,12 @@ public static T makeVectorProcessor( * @param dimensionSpec dimensionSpec for the input to the processor * @param strategyFactory object that encapsulates the knowledge about how to create processors * @param selectorFactory column selector factory used for creating the vector processor + * + * @see ColumnProcessors#makeProcessor the non-vectorized version */ public static T makeVectorProcessor( final DimensionSpec dimensionSpec, - final VectorColumnStrategizer strategyFactory, + final VectorColumnProcessorFactory strategyFactory, final VectorColumnSelectorFactory selectorFactory ) { @@ -285,11 +291,11 @@ public static T makeVectorProcessor( if (type == ValueType.STRING) { if (capabilities.hasMultipleValues()) { - return strategyFactory.makeMultiValueDimensionStrategy( + return strategyFactory.makeMultiValueDimensionProcessor( selectorFactory.makeMultiValueDimensionSelector(dimensionSpec) ); } else { - return strategyFactory.makeSingleValueDimensionStrategy( + return strategyFactory.makeSingleValueDimensionProcessor( selectorFactory.makeSingleValueDimensionSelector(dimensionSpec) ); } @@ -303,15 +309,15 @@ public static T makeVectorProcessor( ); if (type == ValueType.LONG) { - return strategyFactory.makeLongStrategy( + return strategyFactory.makeLongProcessor( selectorFactory.makeValueSelector(dimensionSpec.getDimension()) ); } else if (type == ValueType.FLOAT) { - return strategyFactory.makeFloatStrategy( + return strategyFactory.makeFloatProcessor( selectorFactory.makeValueSelector(dimensionSpec.getDimension()) ); } else if (type == ValueType.DOUBLE) { - return strategyFactory.makeDoubleStrategy( + return strategyFactory.makeDoubleProcessor( selectorFactory.makeValueSelector(dimensionSpec.getDimension()) ); } else { diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index 87faef84d554..13b0541f546e 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -55,6 +55,7 @@ import java.util.Objects; /** + * */ public class QueryableIndexStorageAdapter implements StorageAdapter { @@ -73,12 +74,6 @@ public QueryableIndexStorageAdapter(QueryableIndex index) this.index = index; } - @Override - public String getSegmentIdentifier() - { - throw new UnsupportedOperationException(); - } - @Override public Interval getInterval() { @@ -103,7 +98,8 @@ public int getDimensionCardinality(String dimension) { ColumnHolder columnHolder = index.getColumnHolder(dimension); if (columnHolder == null) { - return 0; + // NullDimensionSelector has cardinality = 1 (one null, nothing else). + return 1; } try (BaseColumn col = columnHolder.getColumn()) { if (!(col instanceof DictionaryEncodedColumn)) { @@ -182,9 +178,15 @@ public ColumnCapabilities getColumnCapabilities(String column) } @Override + @Nullable public String getColumnTypeName(String columnName) { final ColumnHolder columnHolder = index.getColumnHolder(columnName); + + if (columnHolder == null) { + return null; + } + try (final BaseColumn col = columnHolder.getColumn()) { if (col instanceof ComplexColumn) { return ((ComplexColumn) col).getTypeName(); diff --git a/processing/src/main/java/org/apache/druid/segment/RowAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowAdapter.java new file mode 100644 index 000000000000..603d349bd4da --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/RowAdapter.java @@ -0,0 +1,40 @@ +/* + * 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; + +import java.util.function.Function; +import java.util.function.ToLongFunction; + +/** + * An adapter between arbitrary types and the needs of callers that want to read specific columns out of those + * types (treating them as rows). + */ +public interface RowAdapter +{ + /** + * Returns a function that retrieves timestamps from rows. + */ + ToLongFunction timestampFunction(); + + /** + * Returns a function that retrieves the value for column "columnName" from rows. + */ + Function columnFunction(String columnName); +} diff --git a/processing/src/main/java/org/apache/druid/query/groupby/RowBasedColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java similarity index 88% rename from processing/src/main/java/org/apache/druid/query/groupby/RowBasedColumnSelectorFactory.java rename to processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java index b98dcde1167e..74b34003567e 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/RowBasedColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.query.groupby; +package org.apache.druid.segment; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; @@ -28,14 +28,6 @@ import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import org.apache.druid.segment.BaseSingleValueDimensionSelector; -import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.DimensionDictionarySelector; -import org.apache.druid.segment.DimensionHandlerUtils; -import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.IdLookup; -import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; @@ -53,12 +45,20 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory { - public interface RowAdapter + private static final RowAdapter STANDARD_ROW_ADAPTER = new RowAdapter() { - ToLongFunction timestampFunction(); + @Override + public ToLongFunction timestampFunction() + { + return Row::getTimestampFromEpoch; + } - Function rawFunction(String columnName); - } + @Override + public Function columnFunction(String columnName) + { + return r -> r.getRaw(columnName); + } + }; private final Supplier supplier; private final RowAdapter adapter; @@ -80,22 +80,8 @@ public static RowBasedColumnSelectorFactory create( @Nullable final Map signature ) { - final RowAdapter adapter = new RowAdapter() - { - @Override - public ToLongFunction timestampFunction() - { - return Row::getTimestampFromEpoch; - } - - @Override - public Function rawFunction(String columnName) - { - return r -> r.getRaw(columnName); - } - }; - - return new RowBasedColumnSelectorFactory<>(supplier, adapter, signature); + //noinspection unchecked + return new RowBasedColumnSelectorFactory<>(supplier, (RowAdapter) STANDARD_ROW_ADAPTER, signature); } public static RowBasedColumnSelectorFactory create( @@ -107,6 +93,23 @@ public static RowBasedColumnSelectorFactory create( return new RowBasedColumnSelectorFactory<>(supplier, adapter, signature); } + @Nullable + public static ColumnCapabilities getColumnCapabilities( + final Map rowSignature, + final String columnName + ) + { + if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { + // TIME_COLUMN_NAME is handled specially; override the provided rowSignature. + return new ColumnCapabilitiesImpl().setType(ValueType.LONG); + } else { + final ValueType valueType = rowSignature.get(columnName); + + // Do _not_ set isDictionaryEncoded or hasBitmapIndexes, because Row-based columns do not have those things. + return valueType != null ? new ColumnCapabilitiesImpl().setType(valueType) : null; + } + } + @Override public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) { @@ -143,7 +146,7 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) } }; } else { - final Function dimFunction = adapter.rawFunction(dimension); + final Function dimFunction = adapter.columnFunction(dimension); return new DimensionSelector() { @@ -360,20 +363,20 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) } return new TimeLongColumnSelector(); } else { - final Function rawFunction = adapter.rawFunction(columnName); + final Function columnFunction = adapter.columnFunction(columnName); return new ColumnValueSelector() { @Override public boolean isNull() { - return rawFunction.apply(supplier.get()) == null; + return columnFunction.apply(supplier.get()) == null; } @Override public double getDouble() { - Number metric = Rows.objectToNumber(columnName, rawFunction.apply(supplier.get())); + Number metric = Rows.objectToNumber(columnName, columnFunction.apply(supplier.get())); assert NullHandling.replaceWithDefault() || metric != null; return DimensionHandlerUtils.nullToZero(metric).doubleValue(); } @@ -381,7 +384,7 @@ public double getDouble() @Override public float getFloat() { - Number metric = Rows.objectToNumber(columnName, rawFunction.apply(supplier.get())); + Number metric = Rows.objectToNumber(columnName, columnFunction.apply(supplier.get())); assert NullHandling.replaceWithDefault() || metric != null; return DimensionHandlerUtils.nullToZero(metric).floatValue(); } @@ -389,7 +392,7 @@ public float getFloat() @Override public long getLong() { - Number metric = Rows.objectToNumber(columnName, rawFunction.apply(supplier.get())); + Number metric = Rows.objectToNumber(columnName, columnFunction.apply(supplier.get())); assert NullHandling.replaceWithDefault() || metric != null; return DimensionHandlerUtils.nullToZero(metric).longValue(); } @@ -398,7 +401,7 @@ public long getLong() @Override public Object getObject() { - return rawFunction.apply(supplier.get()); + return columnFunction.apply(supplier.get()); } @Override @@ -420,14 +423,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public ColumnCapabilities getColumnCapabilities(String columnName) { - if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { - // TIME_COLUMN_NAME is handled specially; override the provided rowSignature. - return new ColumnCapabilitiesImpl().setType(ValueType.LONG); - } else { - final ValueType valueType = rowSignature.get(columnName); - - // Do _not_ set isDictionaryEncoded or hasBitmapIndexes, because Row-based columns do not have those things. - return valueType != null ? new ColumnCapabilitiesImpl().setType(valueType) : null; - } + return getColumnCapabilities(rowSignature, columnName); } } diff --git a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java index 1dfc00ec12c9..89c2fdbb7f4c 100644 --- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java @@ -32,8 +32,6 @@ @PublicApi public interface StorageAdapter extends CursorFactory { - @PublicApi - String getSegmentIdentifier(); Interval getInterval(); Indexed getAvailableDimensions(); Iterable getAvailableMetrics(); diff --git a/processing/src/main/java/org/apache/druid/query/dimension/VectorColumnStrategizer.java b/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java similarity index 55% rename from processing/src/main/java/org/apache/druid/query/dimension/VectorColumnStrategizer.java rename to processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java index 06338d4b93dd..f76f2dc70e95 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/VectorColumnStrategizer.java +++ b/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.query.dimension; +package org.apache.druid.segment; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; @@ -25,17 +25,24 @@ /** * Class that encapsulates knowledge about how to create vector column processors. Used by - * {@link org.apache.druid.segment.DimensionHandlerUtils#makeVectorProcessor}. + * {@link DimensionHandlerUtils#makeVectorProcessor}. + * + * Unlike {@link ColumnProcessorFactory}, this interface does not have a "defaultType" method. The default type is + * always implicitly STRING. It also does not have a "makeComplexProcessor" method; instead, complex-typed columns + * are fed into "makeSingleValueDimensionProcessor". This behavior may change in the future to better align + * with {@link ColumnProcessorFactory}. + * + * @see ColumnProcessorFactory the non-vectorized version */ -public interface VectorColumnStrategizer +public interface VectorColumnProcessorFactory { - T makeSingleValueDimensionStrategy(SingleValueDimensionVectorSelector selector); + T makeSingleValueDimensionProcessor(SingleValueDimensionVectorSelector selector); - T makeMultiValueDimensionStrategy(MultiValueDimensionVectorSelector selector); + T makeMultiValueDimensionProcessor(MultiValueDimensionVectorSelector selector); - T makeFloatStrategy(VectorValueSelector selector); + T makeFloatProcessor(VectorValueSelector selector); - T makeDoubleStrategy(VectorValueSelector selector); + T makeDoubleProcessor(VectorValueSelector selector); - T makeLongStrategy(VectorValueSelector selector); + T makeLongProcessor(VectorValueSelector selector); } diff --git a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java index 9802aadd661b..b5181b1d2511 100644 --- a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java +++ b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java @@ -141,6 +141,7 @@ public boolean exists(String columnName) return getVirtualColumn(columnName) != null; } + @Nullable public VirtualColumn getVirtualColumn(String columnName) { final VirtualColumn vc = withoutDotSupport.get(columnName); @@ -180,11 +181,18 @@ public BitmapIndex getBitmapIndex(String columnName, ColumnSelector columnSelect if (virtualColumn == null) { throw new IAE("No such virtual column[%s]", columnName); } else { - return virtualColumn.capabilities(columnName).hasBitmapIndexes() ? virtualColumn.getBitmapIndex(columnName, columnSelector) : null; + return virtualColumn.capabilities(columnName).hasBitmapIndexes() ? virtualColumn.getBitmapIndex( + columnName, + columnSelector + ) : null; } } - public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelector columnSelector, ReadableOffset offset) + public DimensionSelector makeDimensionSelector( + DimensionSpec dimensionSpec, + ColumnSelector columnSelector, + ReadableOffset offset + ) { final VirtualColumn virtualColumn = getVirtualColumn(dimensionSpec.getDimension()); if (virtualColumn == null) { @@ -194,7 +202,11 @@ public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, Colu } } - public ColumnValueSelector makeColumnValueSelector(String columnName, ColumnSelector columnSelector, ReadableOffset offset) + public ColumnValueSelector makeColumnValueSelector( + String columnName, + ColumnSelector columnSelector, + ReadableOffset offset + ) { final VirtualColumn virtualColumn = getVirtualColumn(columnName); if (virtualColumn == null) { @@ -226,6 +238,7 @@ public ColumnValueSelector makeColumnValueSelector(String columnName, ColumnS } } + @Nullable public ColumnCapabilities getColumnCapabilities(String columnName) { final VirtualColumn virtualColumn = getVirtualColumn(columnName); @@ -264,7 +277,11 @@ public int size() public ColumnSelectorFactory wrap(final ColumnSelectorFactory baseFactory) { - return new VirtualizedColumnSelectorFactory(baseFactory, this); + if (virtualColumns.isEmpty()) { + return baseFactory; + } else { + return new VirtualizedColumnSelectorFactory(baseFactory, this); + } } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java index 4e1902d87f8d..92dbadb0a677 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java @@ -35,7 +35,7 @@ public interface ColumnCapabilities /** * This property indicates that this {@link ColumnCapabilities} is "complete" in that all properties can be expected * to supply valid responses. Not all {@link ColumnCapabilities} are created equal. Some, such as those provided by - * {@link org.apache.druid.query.groupby.RowBasedColumnSelectorFactory} only have type information, if even that, and + * {@link org.apache.druid.segment.RowBasedColumnSelectorFactory} only have type information, if even that, and * cannot supply information like {@link ColumnCapabilities#hasMultipleValues}, and will report as false. */ boolean isComplete(); diff --git a/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java index f0f8a6b08ad8..ee37e1d59fe1 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java @@ -37,7 +37,7 @@ import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; -import org.apache.druid.query.filter.vector.VectorValueMatcherColumnStrategizer; +import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; @@ -137,7 +137,7 @@ public VectorValueMatcher makeVectorMatcher(final VectorColumnSelectorFactory fa { return DimensionHandlerUtils.makeVectorProcessor( boundDimFilter.getDimension(), - VectorValueMatcherColumnStrategizer.instance(), + VectorValueMatcherColumnProcessorFactory.instance(), factory ).makeMatcher(getPredicateFactory()); } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java index 6029db54e545..50869e6c73ea 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java @@ -34,7 +34,7 @@ import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; -import org.apache.druid.query.filter.vector.VectorValueMatcherColumnStrategizer; +import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionHandlerUtils; @@ -125,7 +125,7 @@ public VectorValueMatcher makeVectorMatcher(final VectorColumnSelectorFactory fa { return DimensionHandlerUtils.makeVectorProcessor( dimension, - VectorValueMatcherColumnStrategizer.instance(), + VectorValueMatcherColumnProcessorFactory.instance(), factory ).makeMatcher(predicateFactory); } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/InFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/InFilter.java index 4594126b37a2..1aec9cc0063a 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/InFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/InFilter.java @@ -36,7 +36,7 @@ import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; -import org.apache.druid.query.filter.vector.VectorValueMatcherColumnStrategizer; +import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionHandlerUtils; @@ -156,7 +156,7 @@ public VectorValueMatcher makeVectorMatcher(final VectorColumnSelectorFactory fa { return DimensionHandlerUtils.makeVectorProcessor( dimension, - VectorValueMatcherColumnStrategizer.instance(), + VectorValueMatcherColumnProcessorFactory.instance(), factory ).makeMatcher(getPredicateFactory()); } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java index 6d148e41c388..aa7f952fccec 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java @@ -33,7 +33,7 @@ import org.apache.druid.query.filter.LikeDimFilter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; -import org.apache.druid.query.filter.vector.VectorValueMatcherColumnStrategizer; +import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionHandlerUtils; @@ -90,7 +90,7 @@ public VectorValueMatcher makeVectorMatcher(final VectorColumnSelectorFactory fa { return DimensionHandlerUtils.makeVectorProcessor( dimension, - VectorValueMatcherColumnStrategizer.instance(), + VectorValueMatcherColumnProcessorFactory.instance(), factory ).makeMatcher(likeMatcher.predicateFactory(extractionFn)); } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/SelectorFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/SelectorFilter.java index 641e768faeff..7d29d0e3feb4 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/SelectorFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/SelectorFilter.java @@ -27,7 +27,7 @@ import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; -import org.apache.druid.query.filter.vector.VectorValueMatcherColumnStrategizer; +import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionHandlerUtils; @@ -81,7 +81,7 @@ public VectorValueMatcher makeVectorMatcher(final VectorColumnSelectorFactory fa { return DimensionHandlerUtils.makeVectorProcessor( dimension, - VectorValueMatcherColumnStrategizer.instance(), + VectorValueMatcherColumnProcessorFactory.instance(), factory ).makeMatcher(value); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java index fd9bcbd0d984..24b640070b7b 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java @@ -48,7 +48,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AbstractIndex; import org.apache.druid.segment.ColumnSelectorFactory; @@ -63,6 +62,7 @@ import org.apache.druid.segment.Metadata; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.ObjectColumnSelector; +import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index b6a27bee1538..2dd613b7cce7 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -61,12 +61,6 @@ public IncrementalIndexStorageAdapter(IncrementalIndex index) this.index = index; } - @Override - public String getSegmentIdentifier() - { - throw new UnsupportedOperationException(); - } - @Override public Interval getInterval() { diff --git a/processing/src/main/java/org/apache/druid/segment/join/Equality.java b/processing/src/main/java/org/apache/druid/segment/join/Equality.java new file mode 100644 index 000000000000..6b839c1f0dc8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/Equality.java @@ -0,0 +1,60 @@ +/* + * 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; + +import org.apache.druid.math.expr.Expr; + +/** + * Represents a join condition between a left-hand-side expression (leftExpr) and a right-hand-side direct + * column access (rightColumn). This is a particularly interesting kind of condition because it can be resolved + * using a hashtable on the right-hand side. + * + * Note that this class does not override "equals" or "hashCode". This is because Expr also does not. + */ +public class Equality +{ + private final Expr leftExpr; + private final String rightColumn; + + public Equality(final Expr leftExpr, final String rightColumn) + { + this.leftExpr = leftExpr; + this.rightColumn = rightColumn; + } + + public Expr getLeftExpr() + { + return leftExpr; + } + + public String getRightColumn() + { + return rightColumn; + } + + @Override + public String toString() + { + return "Equality{" + + "leftExpr=" + leftExpr + + ", rightColumn='" + rightColumn + '\'' + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java new file mode 100644 index 000000000000..71a3526b3544 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -0,0 +1,211 @@ +/* + * 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; + +import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.joda.time.DateTime; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +public class HashJoinEngine +{ + private HashJoinEngine() + { + // No instantiation. + } + + /** + * Creates a cursor that represents the join of {@param leftCursor} with {@param joinableClause}. The resulting + * cursor may generate nulls on the left-hand side (for righty joins; see {@link JoinType#isRighty()}) or on + * the right-hand side (for lefty joins; see {@link JoinType#isLefty()}). Columns that start with the + * joinable clause's prefix (see {@link JoinableClause#getPrefix()}) will come from the Joinable's column selector + * factory, and all other columns will come from the leftCursor's column selector factory. + * + * Ensuing that the joinable clause's prefix does not conflict with any columns from "leftCursor" is the + * responsibility of the caller. + */ + public static Cursor makeJoinCursor(final Cursor leftCursor, final JoinableClause joinableClause) + { + final ColumnSelectorFactory leftColumnSelectorFactory = leftCursor.getColumnSelectorFactory(); + final JoinMatcher joinMatcher = joinableClause.getJoinable() + .makeJoinMatcher( + leftColumnSelectorFactory, + joinableClause.getCondition(), + joinableClause.getJoinType().isRighty() + ); + + class JoinColumnSelectorFactory implements ColumnSelectorFactory + { + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + if (joinableClause.includesColumn(dimensionSpec.getDimension())) { + return joinMatcher.getColumnSelectorFactory() + .makeDimensionSelector( + dimensionSpec.withDimension(joinableClause.unprefix(dimensionSpec.getDimension())) + ); + } else { + final DimensionSelector leftSelector = leftColumnSelectorFactory.makeDimensionSelector(dimensionSpec); + + if (!joinableClause.getJoinType().isRighty()) { + return leftSelector; + } else { + return new PossiblyNullDimensionSelector(leftSelector, joinMatcher::matchingRemainder); + } + } + } + + @Override + public ColumnValueSelector makeColumnValueSelector(String column) + { + if (joinableClause.includesColumn(column)) { + return joinMatcher.getColumnSelectorFactory().makeColumnValueSelector(joinableClause.unprefix(column)); + } else { + final ColumnValueSelector leftSelector = leftColumnSelectorFactory.makeColumnValueSelector(column); + + if (!joinableClause.getJoinType().isRighty()) { + return leftSelector; + } else { + return new PossiblyNullColumnValueSelector<>(leftSelector, joinMatcher::matchingRemainder); + } + } + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + if (joinableClause.includesColumn(column)) { + return joinMatcher.getColumnSelectorFactory().getColumnCapabilities(joinableClause.unprefix(column)); + } else { + return leftColumnSelectorFactory.getColumnCapabilities(column); + } + } + } + + final JoinColumnSelectorFactory joinColumnSelectorFactory = new JoinColumnSelectorFactory(); + + class JoinCursor implements Cursor + { + public void initialize() + { + matchCurrentPosition(); + + if (!joinableClause.getJoinType().isLefty()) { + while (!joinMatcher.hasMatch() && !isDone()) { + advance(); + matchCurrentPosition(); + } + } + } + + @Override + @Nonnull + public ColumnSelectorFactory getColumnSelectorFactory() + { + return joinColumnSelectorFactory; + } + + @Override + @Nonnull + public DateTime getTime() + { + return leftCursor.getTime(); + } + + @Override + public void advance() + { + advanceUninterruptibly(); + BaseQuery.checkInterrupted(); + } + + private void matchCurrentPosition() + { + if (leftCursor.isDone()) { + if (joinableClause.getJoinType().isRighty() && !joinMatcher.matchingRemainder()) { + // Warning! The way this engine handles "righty" joins is flawed: it generates the 'remainder' rows + // per-segment, but this should really be done globally. This should be improved in the future. + joinMatcher.matchRemainder(); + } + } else { + joinMatcher.matchCondition(); + } + } + + @Override + public void advanceUninterruptibly() + { + if (joinMatcher.hasMatch()) { + joinMatcher.nextMatch(); + + if (joinMatcher.hasMatch()) { + return; + } + } + + assert !joinMatcher.hasMatch(); + + do { + // No more right-hand side matches; advance the left-hand side. + leftCursor.advanceUninterruptibly(); + + // Update joinMatcher state to match new cursor position. + matchCurrentPosition(); + + // If this is not a left/full join, and joinMatcher didn't match anything, then keep advancing until we find + // left rows that have matching right rows. + } while (!joinableClause.getJoinType().isLefty() + && !joinMatcher.hasMatch() + && !leftCursor.isDone()); + } + + @Override + public boolean isDone() + { + return leftCursor.isDone() && !joinMatcher.hasMatch(); + } + + @Override + public boolean isDoneOrInterrupted() + { + return isDone() || Thread.currentThread().isInterrupted(); + } + + @Override + public void reset() + { + leftCursor.reset(); + joinMatcher.reset(); + } + } + + final JoinCursor joinCursor = new JoinCursor(); + joinCursor.initialize(); + return joinCursor; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java new file mode 100644 index 000000000000..e35f550577d5 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.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.druid.segment.join; + +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.AbstractSegment; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.List; + +/** + * Represents a deep, left-heavy join of a left-hand side baseSegment onto a series of right-hand side clauses. + * + * In other words, logically the operation is: join(join(join(baseSegment, clauses[0]), clauses[1]), clauses[2]) etc. + */ +public class HashJoinSegment extends AbstractSegment +{ + private final Segment baseSegment; + private final List clauses; + + public HashJoinSegment( + Segment baseSegment, + List clauses + ) + { + this.baseSegment = baseSegment; + this.clauses = clauses; + + // Verify no clauses would shadow the special __time field. + for (JoinableClause clause : clauses) { + if (clause.includesColumn(ColumnHolder.TIME_COLUMN_NAME)) { + throw new IAE( + "Clause cannot have prefix[%s], since it would shadow %s", + clause.getPrefix(), + ColumnHolder.TIME_COLUMN_NAME + ); + } + } + } + + @Override + public SegmentId getId() + { + return baseSegment.getId(); + } + + @Override + public Interval getDataInterval() + { + // __time column will come from the baseSegment, so use its data interval. + return baseSegment.getDataInterval(); + } + + @Nullable + @Override + public QueryableIndex asQueryableIndex() + { + // Even if baseSegment is a QueryableIndex, we don't want to expose it, since we've modified its behavior + // too much while wrapping it. + return null; + } + + @Override + public StorageAdapter asStorageAdapter() + { + return new HashJoinSegmentStorageAdapter(baseSegment.asStorageAdapter(), clauses); + } + + @Override + public void close() throws IOException + { + baseSegment.close(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java new file mode 100644 index 000000000000..72dc0f551ccc --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -0,0 +1,279 @@ +/* + * 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; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.QueryMetrics; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.segment.Capabilities; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.Metadata; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.data.ListIndexed; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +public class HashJoinSegmentStorageAdapter implements StorageAdapter +{ + private final StorageAdapter baseAdapter; + private final List clauses; + + HashJoinSegmentStorageAdapter( + StorageAdapter baseAdapter, + List clauses + ) + { + this.baseAdapter = baseAdapter; + this.clauses = clauses; + } + + @Override + public Interval getInterval() + { + return baseAdapter.getInterval(); + } + + @Override + public Indexed getAvailableDimensions() + { + // Use a Set since we may encounter duplicates, if a field from a Joinable shadows one of the base fields. + final LinkedHashSet availableDimensions = new LinkedHashSet<>(); + + baseAdapter.getAvailableDimensions().forEach(availableDimensions::add); + + for (JoinableClause clause : clauses) { + availableDimensions.addAll(clause.getAvailableColumnsPrefixed()); + } + + return new ListIndexed<>(Lists.newArrayList(availableDimensions)); + } + + @Override + public Iterable getAvailableMetrics() + { + return baseAdapter.getAvailableMetrics(); + } + + @Override + public int getDimensionCardinality(String column) + { + final Optional maybeClause = getClauseForColumn(column); + + if (maybeClause.isPresent()) { + final JoinableClause clause = maybeClause.get(); + return clause.getJoinable().getCardinality(clause.unprefix(column)); + } else { + return baseAdapter.getDimensionCardinality(column); + } + } + + @Override + public DateTime getMinTime() + { + return baseAdapter.getMinTime(); + } + + @Override + public DateTime getMaxTime() + { + return baseAdapter.getMaxTime(); + } + + @Nullable + @Override + public Comparable getMinValue(String column) + { + if (isBaseColumn(column)) { + return baseAdapter.getMinValue(column); + } else { + return null; + } + } + + @Nullable + @Override + public Comparable getMaxValue(String column) + { + if (isBaseColumn(column)) { + return baseAdapter.getMaxValue(column); + } else { + return null; + } + } + + @Override + public Capabilities getCapabilities() + { + // Dictionaries in the joinables may not be sorted. Unfortunately this API does not let us be granular about what + // is and isn't sorted, so return false globally. At the time of this writing, the only query affected by this + // is a topN with lexicographic sort and 'previousStop' set (it will not be able to skip values based on + // dictionary code). + return Capabilities.builder().dimensionValuesSorted(false).build(); + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + final Optional maybeClause = getClauseForColumn(column); + + if (maybeClause.isPresent()) { + final JoinableClause clause = maybeClause.get(); + return clause.getJoinable().getColumnCapabilities(clause.unprefix(column)); + } else { + return baseAdapter.getColumnCapabilities(column); + } + } + + @Nullable + @Override + public String getColumnTypeName(String column) + { + final Optional maybeClause = getClauseForColumn(column); + + if (maybeClause.isPresent()) { + final JoinableClause clause = maybeClause.get(); + final ColumnCapabilities capabilities = clause.getJoinable().getColumnCapabilities(clause.unprefix(column)); + return capabilities != null ? capabilities.getType().toString() : null; + } else { + return baseAdapter.getColumnTypeName(column); + } + } + + @Override + public int getNumRows() + { + // Cannot determine the number of rows ahead of time for a join segment (rows may be added or removed based + // on the join condition). At the time of this writing, this method is only used by the 'segmentMetadata' query, + // which isn't meant to support join segments anyway. + throw new UnsupportedOperationException("Cannot retrieve number of rows from join segment"); + } + + @Override + public DateTime getMaxIngestedEventTime() + { + return baseAdapter.getMaxIngestedEventTime(); + } + + @Override + public Metadata getMetadata() + { + // Cannot get meaningful Metadata for this segment, since it isn't real. At the time of this writing, this method + // is only used by the 'segmentMetadata' query, which isn't meant to support join segments anyway. + throw new UnsupportedOperationException("Cannot retrieve metadata from join segment"); + } + + @Override + public Sequence makeCursors( + @Nullable final Filter filter, + @Nonnull final Interval interval, + @Nonnull final VirtualColumns virtualColumns, + @Nonnull final Granularity gran, + final boolean descending, + @Nullable final QueryMetrics queryMetrics + ) + { + final Set baseColumns = new HashSet<>(); + Iterables.addAll(baseColumns, baseAdapter.getAvailableDimensions()); + Iterables.addAll(baseColumns, baseAdapter.getAvailableMetrics()); + + final List preJoinVirtualColumns = new ArrayList<>(); + final List postJoinVirtualColumns = new ArrayList<>(); + + for (VirtualColumn virtualColumn : virtualColumns.getVirtualColumns()) { + // Virtual columns cannot depend on each other, so we don't need to check transitive dependencies. + if (baseColumns.containsAll(virtualColumn.requiredColumns())) { + preJoinVirtualColumns.add(virtualColumn); + } else { + postJoinVirtualColumns.add(virtualColumn); + } + } + + // Soon, we will need a way to push filters past a join when possible. This could potentially be done right here + // (by splitting out pushable pieces of 'filter') or it could be done at a higher level (i.e. in the SQL planner). + // + // If it's done in the SQL planner, that will likely mean adding a 'baseFilter' parameter to this class that would + // be passed in to the below baseAdapter.makeCursors call (instead of the null filter). + final Sequence baseCursorSequence = baseAdapter.makeCursors( + null, + interval, + VirtualColumns.create(preJoinVirtualColumns), + gran, + descending, + queryMetrics + ); + + return Sequences.map( + baseCursorSequence, + cursor -> { + Cursor retVal = cursor; + + for (JoinableClause clause : clauses) { + retVal = HashJoinEngine.makeJoinCursor(retVal, clause); + } + + return PostJoinCursor.wrap(retVal, VirtualColumns.create(postJoinVirtualColumns), filter); + } + ); + } + + /** + * Returns whether "column" will be selected from "baseAdapter". This is true if it is not shadowed by any joinables + * (i.e. if it does not start with any of their prefixes). + */ + private boolean isBaseColumn(final String column) + { + return !getClauseForColumn(column).isPresent(); + } + + /** + * Returns the JoinableClause corresponding to a particular column, based on the clauses' prefixes. + * + * @param column column name + * + * @return the clause, or absent if the column does not correspond to any clause + */ + private Optional getClauseForColumn(final String column) + { + // Check clauses in reverse, since "makeCursors" creates the cursor in such a way that the last clause + // gets first dibs to claim a column. + return Lists.reverse(clauses) + .stream() + .filter(clause -> clause.includesColumn(column)) + .findFirst(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinConditionAnalysis.java b/processing/src/main/java/org/apache/druid/segment/join/JoinConditionAnalysis.java new file mode 100644 index 000000000000..dc643401d778 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinConditionAnalysis.java @@ -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; + +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.math.expr.Expr; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.math.expr.Exprs; +import org.apache.druid.math.expr.Parser; +import org.apache.druid.query.expression.ExprUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * Represents analysis of a join condition. + * + * Each condition is decomposed into "equiConditions" and "nonEquiConditions". + * + * 1) The equiConditions are of the form ExpressionOfLeft = ColumnFromRight. The right-hand part cannot be an expression + * because we use this analysis to determine if we can perform the join using hashtables built off right-hand-side + * columns. + * + * 2) The nonEquiConditions are other conditions that should also be ANDed together + * + * All of these conditions are ANDed together to get the overall condition. + */ +public class JoinConditionAnalysis +{ + private final String originalExpression; + private final List equiConditions; + private final List nonEquiConditions; + + private JoinConditionAnalysis( + final String originalExpression, + final List equiConditions, + final List nonEquiConditions + ) + { + this.originalExpression = Preconditions.checkNotNull(originalExpression, "originalExpression"); + this.equiConditions = equiConditions; + this.nonEquiConditions = nonEquiConditions; + } + + public static JoinConditionAnalysis forExpression( + final String condition, + final String rightPrefix, + final ExprMacroTable macroTable + ) + { + final Expr conditionExpr = Parser.parse(condition, macroTable); + final List equiConditions = new ArrayList<>(); + final List nonEquiConditions = new ArrayList<>(); + + final List exprs = Exprs.decomposeAnd(conditionExpr); + for (Expr childExpr : exprs) { + final Optional> maybeDecomposed = Exprs.decomposeEquals(childExpr); + + if (!maybeDecomposed.isPresent()) { + nonEquiConditions.add(childExpr); + } else { + final Pair decomposed = maybeDecomposed.get(); + final Expr lhs = decomposed.lhs; + final Expr rhs = decomposed.rhs; + + if (isLeftExprAndRightColumn(lhs, rhs, rightPrefix)) { + // rhs is a right-hand column; lhs is an expression solely of the left-hand side. + equiConditions.add(new Equality(lhs, rhs.getIdentifierIfIdentifier().substring(rightPrefix.length()))); + } else if (isLeftExprAndRightColumn(rhs, lhs, rightPrefix)) { + equiConditions.add(new Equality(rhs, lhs.getIdentifierIfIdentifier().substring(rightPrefix.length()))); + } else { + nonEquiConditions.add(childExpr); + } + } + } + + return new JoinConditionAnalysis(condition, equiConditions, nonEquiConditions); + } + + private static boolean isLeftExprAndRightColumn(final Expr a, final Expr b, final String rightPrefix) + { + return a.analyzeInputs().getRequiredBindings().stream().noneMatch(c -> c.startsWith(rightPrefix)) + && b.getIdentifierIfIdentifier() != null + && b.getIdentifierIfIdentifier().startsWith(rightPrefix); + } + + /** + * Return the condition expression. + */ + public String getOriginalExpression() + { + return originalExpression; + } + + /** + * Return a list of equi-conditions (see class-level javadoc). + */ + public List getEquiConditions() + { + return equiConditions; + } + + /** + * Return a list of non-equi-conditions (see class-level javadoc). + */ + public List getNonEquiConditions() + { + return nonEquiConditions; + } + + /** + * Return whether this condition is a constant that is always false. + */ + public boolean isAlwaysFalse() + { + return nonEquiConditions.stream() + .anyMatch(expr -> expr.isLiteral() && !expr.eval(ExprUtils.nilBindings()).asBoolean()); + } + + /** + * Return whether this condition is a constant that is always true. + */ + public boolean isAlwaysTrue() + { + return equiConditions.isEmpty() && + nonEquiConditions.stream() + .allMatch(expr -> expr.isLiteral() && expr.eval(ExprUtils.nilBindings()).asBoolean()); + } + + /** + * Returns whether this condition can be satisfied using a hashtable made from the right-hand side. + */ + public boolean canHashJoin() + { + return nonEquiConditions.stream().allMatch(Expr::isLiteral); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JoinConditionAnalysis that = (JoinConditionAnalysis) o; + return Objects.equals(originalExpression, that.originalExpression); + } + + @Override + public int hashCode() + { + return Objects.hash(originalExpression); + } + + @Override + public String toString() + { + return originalExpression; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinMatcher.java b/processing/src/main/java/org/apache/druid/segment/join/JoinMatcher.java new file mode 100644 index 000000000000..f5f3f4c27915 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinMatcher.java @@ -0,0 +1,83 @@ +/* + * 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; + +import org.apache.druid.segment.ColumnSelectorFactory; + +/** + * An interface returned by {@link Joinable#makeJoinMatcher} and used by {@link HashJoinEngine} to implement a join. + * + * A typical usage would go something like: + * + *
+ * matcher.matchCondition();
+ * while (matcher.hasMatch()) {
+ *   // Do something with the match
+ *   matcher.nextMatch();
+ * }
+ * 
+ */ +public interface JoinMatcher +{ + /** + * Returns a factory for reading columns from the {@link Joinable} that correspond to matched rows. + */ + ColumnSelectorFactory getColumnSelectorFactory(); + + /** + * Matches against the {@link ColumnSelectorFactory} and {@link JoinConditionAnalysis} supplied to + * {@link Joinable#makeJoinMatcher}. + * + * After calling this method, {@link #hasMatch()} will return whether at least one row matched. After reading that + * row, {@link #nextMatch()} can be used to move on to the next row. + */ + void matchCondition(); + + /** + * Matches every row that has not already been matched. Used for right joins. + * + * After calling this method, {@link #hasMatch()} will return whether at least one row matched. After reading that + * row, {@link #nextMatch()} can be used to move on to the next row. + * + * Will only work correctly if {@link Joinable#makeJoinMatcher} was called with {@code remainderNeeded == true}. + */ + void matchRemainder(); + + /** + * Returns whether the active matcher ({@link #matchCondition()} or {@link #matchRemainder()}) has matched something. + */ + boolean hasMatch(); + + /** + * Moves on to the next match. It is only valid to call this if {@link #hasMatch()} is true. + */ + void nextMatch(); + + /** + * Returns whether this matcher is currently matching the remainder (i.e. if {@link #matchRemainder()} was the + * most recent match method called). + */ + boolean matchingRemainder(); + + /** + * Clears any active matches. Does not clear memory about what has been matched in the past. + */ + void reset(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinType.java b/processing/src/main/java/org/apache/druid/segment/join/JoinType.java new file mode 100644 index 000000000000..e628d7d0d471 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinType.java @@ -0,0 +1,89 @@ +/* + * 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; + +public enum JoinType +{ + INNER { + @Override + boolean isLefty() + { + return false; + } + + @Override + boolean isRighty() + { + return false; + } + }, + + LEFT { + @Override + boolean isLefty() + { + return true; + } + + @Override + boolean isRighty() + { + return false; + } + }, + + RIGHT { + @Override + boolean isLefty() + { + return false; + } + + @Override + boolean isRighty() + { + return true; + } + }, + + FULL { + @Override + boolean isLefty() + { + return true; + } + + @Override + boolean isRighty() + { + return true; + } + }; + + /** + * "Lefty" joins (LEFT or FULL) always include the full left-hand side, and can generate nulls on the right. + */ + abstract boolean isLefty(); + + /** + * "Righty" joins (RIGHT or FULL) always include the full right-hand side, and can generate nulls on the left. + */ + abstract boolean isRighty(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/Joinable.java b/processing/src/main/java/org/apache/druid/segment/join/Joinable.java new file mode 100644 index 000000000000..bc7b6ad678c8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/Joinable.java @@ -0,0 +1,74 @@ +/* + * 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; + +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnCapabilities; + +import javax.annotation.Nullable; +import java.util.List; + +/** + * Represents something that can be the right-hand side of a join. + * + * This class's most important method is {@link #makeJoinMatcher}. Its main user is + * {@link HashJoinEngine#makeJoinCursor}. + */ +public interface Joinable +{ + int CARDINALITY_UNKNOWN = -1; + + /** + * Returns the list of columns that this Joinable offers. + */ + List getAvailableColumns(); + + /** + * Returns the cardinality of "columnName", or {@link #CARDINALITY_UNKNOWN} if not known. May be used at query + * time to trigger optimizations. + */ + int getCardinality(String columnName); + + /** + * Returns capabilities for one of this Joinable's columns. + * + * @param columnName column name + * + * @return capabilities, or null if the columnName is not one of this Joinable's columns + */ + @Nullable + ColumnCapabilities getColumnCapabilities(String columnName); + + /** + * Creates a JoinMatcher that can be used to implement a join onto this Joinable. + * + * @param leftColumnSelectorFactory column selector factory that allows access to the left-hand side of the join + * @param condition join condition for the matcher + * @param remainderNeeded whether or not {@link JoinMatcher#matchRemainder()} will ever be called on the + * matcher. If we know it will not, additional optimizations are often possible. + * + * @return the matcher + */ + JoinMatcher makeJoinMatcher( + ColumnSelectorFactory leftColumnSelectorFactory, + JoinConditionAnalysis condition, + boolean remainderNeeded + ); +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java b/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java new file mode 100644 index 000000000000..21d75f3af3e0 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java @@ -0,0 +1,145 @@ +/* + * 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; + +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.IAE; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Represents everything about a join clause except for the left-hand datasource. In other words, if the full join + * clause is "t1 JOIN t2 ON t1.x = t2.x" then this class represents "JOIN t2 ON x = t2.x" -- it does not include + * references to the left-hand "t1". + */ +public class JoinableClause +{ + private final String prefix; + private final Joinable joinable; + private final JoinType joinType; + private final JoinConditionAnalysis condition; + + public JoinableClause(@Nullable String prefix, Joinable joinable, JoinType joinType, JoinConditionAnalysis condition) + { + this.prefix = prefix != null ? prefix : ""; + this.joinable = Preconditions.checkNotNull(joinable, "joinable"); + this.joinType = Preconditions.checkNotNull(joinType, "joinType"); + this.condition = Preconditions.checkNotNull(condition, "condition"); + } + + /** + * The prefix to apply to all columns from the Joinable. + */ + public String getPrefix() + { + return prefix; + } + + /** + * The right-hand Joinable. + */ + public Joinable getJoinable() + { + return joinable; + } + + /** + * The type of join: LEFT, RIGHT, INNER, or FULL. + */ + public JoinType getJoinType() + { + return joinType; + } + + /** + * The join condition. When referring to right-hand columns, it should include the prefix. + */ + public JoinConditionAnalysis getCondition() + { + return condition; + } + + /** + * Returns a list of columns from the underlying {@link Joinable#getAvailableColumns()} method, with our + * prefix ({@link #getPrefix()}) prepended. + */ + public List getAvailableColumnsPrefixed() + { + return joinable.getAvailableColumns().stream().map(columnName -> prefix + columnName).collect(Collectors.toList()); + } + + /** + * Returns whether "columnName" can be retrieved from the {@link Joinable} represented by this clause (i.e., whether + * it starts with {@code prefix} and has at least one other character beyond that). + */ + public boolean includesColumn(final String columnName) + { + return columnName.startsWith(prefix) && columnName.length() > prefix.length(); + } + + /** + * Removes our prefix from "columnName". Must only be called if {@link #includesColumn} would have returned true + * on this column name. + */ + public String unprefix(final String columnName) + { + if (includesColumn(columnName)) { + return columnName.substring(prefix.length()); + } else { + throw new IAE("Column[%s] does not start with prefix[%s]", columnName, prefix); + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JoinableClause that = (JoinableClause) o; + return Objects.equals(prefix, that.prefix) && + Objects.equals(joinable, that.joinable) && + joinType == that.joinType && + Objects.equals(condition, that.condition); + } + + @Override + public int hashCode() + { + return Objects.hash(prefix, joinable, joinType, condition); + } + + @Override + public String toString() + { + return "JoinableClause{" + + "prefix='" + prefix + '\'' + + ", joinable=" + joinable + + ", joinType=" + joinType + + ", condition=" + condition + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/PossiblyNullColumnValueSelector.java b/processing/src/main/java/org/apache/druid/segment/join/PossiblyNullColumnValueSelector.java new file mode 100644 index 000000000000..b42a048135f9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/PossiblyNullColumnValueSelector.java @@ -0,0 +1,86 @@ +/* + * 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; + +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; +import java.util.function.BooleanSupplier; + +public class PossiblyNullColumnValueSelector implements ColumnValueSelector +{ + private final ColumnValueSelector baseSelector; + private final BooleanSupplier beNull; + @Nullable + private final T nullValue; + + PossiblyNullColumnValueSelector(final ColumnValueSelector baseSelector, final BooleanSupplier beNull) + { + this.baseSelector = baseSelector; + this.beNull = beNull; + this.nullValue = NullHandling.defaultValueForClass(baseSelector.classOfObject()); + } + + @Override + public double getDouble() + { + return beNull.getAsBoolean() ? NullHandling.ZERO_DOUBLE : baseSelector.getDouble(); + } + + @Override + public float getFloat() + { + return beNull.getAsBoolean() ? NullHandling.ZERO_FLOAT : baseSelector.getFloat(); + } + + @Override + public long getLong() + { + return beNull.getAsBoolean() ? NullHandling.ZERO_LONG : baseSelector.getLong(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + baseSelector.inspectRuntimeShape(inspector); + inspector.visit("beNull", beNull); + } + + @Nullable + @Override + public T getObject() + { + return beNull.getAsBoolean() ? nullValue : baseSelector.getObject(); + } + + @Override + public Class classOfObject() + { + return baseSelector.classOfObject(); + } + + @Override + public boolean isNull() + { + return beNull.getAsBoolean() || baseSelector.isNull(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/PossiblyNullDimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/join/PossiblyNullDimensionSelector.java new file mode 100644 index 000000000000..b678b45a4c13 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/PossiblyNullDimensionSelector.java @@ -0,0 +1,191 @@ +/* + * 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; + +import com.google.common.base.Predicate; +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.AbstractDimensionSelector; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.DimensionSelectorUtils; +import org.apache.druid.segment.IdLookup; +import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.data.ZeroIndexedInts; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.function.BooleanSupplier; + +public class PossiblyNullDimensionSelector extends AbstractDimensionSelector implements IdLookup +{ + private final DimensionSelector baseSelector; + private final BooleanSupplier beNull; + private final NullAdjustedIndexedInts nullAdjustedRow; + + // 1 for selectors that have dictionaries (cardinality >= 0), but don't contain null. + private final int nullAdjustment; + + public PossiblyNullDimensionSelector(DimensionSelector baseSelector, BooleanSupplier beNull) + { + this.baseSelector = baseSelector; + this.beNull = beNull; + + if (baseSelector.nameLookupPossibleInAdvance() && + (baseSelector.getValueCardinality() == 0 || (baseSelector.getValueCardinality() > 0 + && baseSelector.lookupName(0) != null))) { + this.nullAdjustment = 1; + } else { + this.nullAdjustment = 0; + } + + this.nullAdjustedRow = new NullAdjustedIndexedInts(nullAdjustment); + } + + @Override + @Nonnull + public IndexedInts getRow() + { + if (beNull.getAsBoolean()) { + // This is the reason we have all the nullAdjustment business. We need to return a null when asked to. + return ZeroIndexedInts.instance(); + } else { + nullAdjustedRow.set(baseSelector.getRow()); + return nullAdjustedRow; + } + } + + @Override + @Nonnull + public ValueMatcher makeValueMatcher(@Nullable final String value) + { + return DimensionSelectorUtils.makeValueMatcherGeneric(this, value); + } + + @Override + @Nonnull + public ValueMatcher makeValueMatcher(final Predicate predicate) + { + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + } + + @Override + public int getValueCardinality() + { + return baseSelector.getValueCardinality() + nullAdjustment; + } + + @Nullable + @Override + public String lookupName(int id) + { + final int cardinality = getValueCardinality(); + + if (cardinality == CARDINALITY_UNKNOWN) { + // CARDINALITY_UNKNOWN means lookupName is only being called in the context of a single row, + // so it's safe to look at "beNull" here. + if (beNull.getAsBoolean()) { + assert id == 0; + return null; + } else { + return baseSelector.lookupName(id - nullAdjustment); + } + } else { + assert cardinality > 0; + + if (id == 0) { + // id 0 is always null for this selector impl. + return null; + } else { + return baseSelector.lookupName(id - nullAdjustment); + } + } + } + + @Override + public boolean nameLookupPossibleInAdvance() + { + return baseSelector.nameLookupPossibleInAdvance(); + } + + @Nullable + @Override + public IdLookup idLookup() + { + return baseSelector.idLookup() != null ? this : null; + } + + @Override + public int lookupId(@Nullable String name) + { + if (name == null) { + // id 0 is always null for this selector impl. + return 0; + } else { + return baseSelector.idLookup().lookupId(name) + nullAdjustment; + } + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("beNull", beNull); + baseSelector.inspectRuntimeShape(inspector); + } + + @Override + public Class classOfObject() + { + return baseSelector.classOfObject(); + } + + private static class NullAdjustedIndexedInts implements IndexedInts + { + private final int nullAdjustment; + private IndexedInts ints = null; + + public NullAdjustedIndexedInts(int nullAdjustment) + { + this.nullAdjustment = nullAdjustment; + } + + public void set(IndexedInts ints) + { + this.ints = ints; + } + + @Override + public int size() + { + return ints.size(); + } + + @Override + public int get(int index) + { + return ints.get(index) + nullAdjustment; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("ints", ints); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java new file mode 100644 index 000000000000..d003da0bb320 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -0,0 +1,121 @@ +/* + * 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; + +import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.VirtualColumns; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; + +/** + * A Cursor decorator used by {@link HashJoinSegmentStorageAdapter#makeCursors} to add post-join virtual columns + * and filters. + */ +public class PostJoinCursor implements Cursor +{ + private final Cursor baseCursor; + private final ColumnSelectorFactory columnSelectorFactory; + + @Nullable + private final ValueMatcher valueMatcher; + + private PostJoinCursor(Cursor baseCursor, VirtualColumns virtualColumns, @Nullable Filter filter) + { + this.baseCursor = baseCursor; + + this.columnSelectorFactory = virtualColumns.wrap(baseCursor.getColumnSelectorFactory()); + + if (filter == null) { + this.valueMatcher = null; + } else { + this.valueMatcher = filter.makeMatcher(this.columnSelectorFactory); + } + } + + public static PostJoinCursor wrap( + final Cursor baseCursor, + final VirtualColumns virtualColumns, + @Nullable final Filter filter + ) + { + final PostJoinCursor postJoinCursor = new PostJoinCursor(baseCursor, virtualColumns, filter); + postJoinCursor.advanceToMatch(); + return postJoinCursor; + } + + private void advanceToMatch() + { + if (valueMatcher != null) { + while (!isDone() && !valueMatcher.matches()) { + baseCursor.advanceUninterruptibly(); + } + } + } + + @Override + public ColumnSelectorFactory getColumnSelectorFactory() + { + return columnSelectorFactory; + } + + @Override + public DateTime getTime() + { + return baseCursor.getTime(); + } + + @Override + public void advance() + { + advanceUninterruptibly(); + BaseQuery.checkInterrupted(); + } + + @Override + public void advanceUninterruptibly() + { + baseCursor.advanceUninterruptibly(); + advanceToMatch(); + } + + @Override + public boolean isDone() + { + return baseCursor.isDone(); + } + + @Override + public boolean isDoneOrInterrupted() + { + return baseCursor.isDoneOrInterrupted(); + } + + @Override + public void reset() + { + baseCursor.reset(); + advanceToMatch(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupColumnSelectorFactory.java new file mode 100644 index 000000000000..845b5cb45623 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupColumnSelectorFactory.java @@ -0,0 +1,113 @@ +/* + * 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.lookup; + +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.BaseSingleValueDimensionSelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ValueType; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.function.Supplier; + +public class LookupColumnSelectorFactory implements ColumnSelectorFactory +{ + public static final String KEY_COLUMN = "k"; + public static final String VALUE_COLUMN = "v"; + + private final Supplier> currentEntry; + + LookupColumnSelectorFactory( + final Supplier> currentEntry + ) + { + this.currentEntry = currentEntry; + } + + @Nonnull + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + final Supplier supplierToUse; + + if (KEY_COLUMN.equals(dimensionSpec.getDimension())) { + supplierToUse = () -> { + final Pair entry = currentEntry.get(); + return entry != null ? entry.lhs : null; + }; + } else if (VALUE_COLUMN.equals(dimensionSpec.getDimension())) { + supplierToUse = () -> { + final Pair entry = currentEntry.get(); + return entry != null ? entry.rhs : null; + }; + } else { + return DimensionSelector.constant(null, dimensionSpec.getExtractionFn()); + } + + return dimensionSpec.decorate( + new BaseSingleValueDimensionSelector() + { + @Nullable + @Override + protected String getValue() + { + if (dimensionSpec.getExtractionFn() != null) { + return dimensionSpec.getExtractionFn().apply(supplierToUse.get()); + } else { + return supplierToUse.get(); + } + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("dimensionSpec", dimensionSpec); + inspector.visit("supplier", supplierToUse); + } + } + ); + } + + @Nonnull + @Override + public ColumnValueSelector makeColumnValueSelector(String columnName) + { + return makeDimensionSelector(DefaultDimensionSpec.of(columnName)); + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + if (LookupJoinable.ALL_COLUMNS.contains(column)) { + return new ColumnCapabilitiesImpl().setType(ValueType.STRING); + } else { + return null; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinMatcher.java b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinMatcher.java new file mode 100644 index 000000000000..9af6df93fdc6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinMatcher.java @@ -0,0 +1,312 @@ +/* + * 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.lookup; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterators; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.common.guava.SettableSupplier; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.math.expr.Expr; +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.BaseDoubleColumnValueSelector; +import org.apache.druid.segment.BaseFloatColumnValueSelector; +import org.apache.druid.segment.BaseLongColumnValueSelector; +import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.ColumnProcessorFactory; +import org.apache.druid.segment.ColumnProcessors; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.join.Equality; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.JoinMatcher; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +public class LookupJoinMatcher implements JoinMatcher +{ + private static final ColumnProcessorFactory> LEFT_KEY_READER = + new ColumnProcessorFactory>() + { + @Override + public ValueType defaultType() + { + return ValueType.STRING; + } + + @Override + public Supplier makeDimensionProcessor(DimensionSelector selector) + { + return () -> { + final IndexedInts row = selector.getRow(); + + if (row.size() == 1) { + return selector.lookupName(row.get(0)); + } else { + // Multi-valued rows are not handled by the join system right now; treat them as nulls. + return null; + } + }; + } + + @Override + public Supplier makeFloatProcessor(BaseFloatColumnValueSelector selector) + { + if (NullHandling.replaceWithDefault()) { + return () -> DimensionHandlerUtils.convertObjectToString(selector.getFloat()); + } else { + return () -> selector.isNull() ? null : DimensionHandlerUtils.convertObjectToString(selector.getFloat()); + } + } + + @Override + public Supplier makeDoubleProcessor(BaseDoubleColumnValueSelector selector) + { + if (NullHandling.replaceWithDefault()) { + return () -> DimensionHandlerUtils.convertObjectToString(selector.getDouble()); + } else { + return () -> selector.isNull() ? null : DimensionHandlerUtils.convertObjectToString(selector.getDouble()); + } + } + + @Override + public Supplier makeLongProcessor(BaseLongColumnValueSelector selector) + { + if (NullHandling.replaceWithDefault()) { + return () -> DimensionHandlerUtils.convertObjectToString(selector.getLong()); + } else { + return () -> selector.isNull() ? null : DimensionHandlerUtils.convertObjectToString(selector.getLong()); + } + } + + @Override + public Supplier makeComplexProcessor(BaseObjectColumnValueSelector selector) + { + return () -> null; + } + }; + + // currentIterator and currentEntry track iteration position through the currently-matched-rows. + // 1) currentEntry is the entry that our column selector factory is looking at right now. + // 2) currentIterator contains future matches that it _will_ be looking at after nextMatch() is called. + @Nullable + private Iterator> currentIterator = null; + private final SettableSupplier> currentEntry = new SettableSupplier<>(); + + private final LookupExtractor extractor; + private final JoinConditionAnalysis condition; + private final List> keySuppliers; + private final ColumnSelectorFactory selectorFactory = new LookupColumnSelectorFactory(currentEntry::get); + + // matchedKeys and matchingRemainder are used to implement matchRemainder(). + private boolean matchingRemainder = false; + private final Set matchedKeys; + + private LookupJoinMatcher( + LookupExtractor extractor, + ColumnSelectorFactory leftSelectorFactory, + JoinConditionAnalysis condition, + @Nullable List keyExprs, + boolean remainderNeeded + ) + { + this.extractor = extractor; + this.matchedKeys = remainderNeeded && !condition.isAlwaysTrue() && !condition.isAlwaysFalse() + ? new HashSet<>() + : null; + this.condition = condition; + + if (keyExprs != null) { + this.keySuppliers = keyExprs.stream() + .map( + expr -> + ColumnProcessors.makeProcessor( + expr, + ValueType.STRING, + LEFT_KEY_READER, + leftSelectorFactory + ) + ) + .collect(Collectors.toList()); + } else { + // This check is to guard against bugs; users should never see it. + Preconditions.checkState( + condition.isAlwaysFalse() || condition.isAlwaysTrue(), + "Condition must be always true or always false when keySuppliers == null" + ); + + this.keySuppliers = null; + } + + // Verify that extractor can be iterated when needed. + if (condition.isAlwaysTrue() || remainderNeeded) { + Preconditions.checkState( + extractor.canIterate(), + "Cannot iterate lookup, but iteration is required for this join" + ); + } + } + + public static LookupJoinMatcher create( + LookupExtractor extractor, + ColumnSelectorFactory leftSelectorFactory, + JoinConditionAnalysis condition, + boolean remainderNeeded + ) + { + final List keyExprs; + + if (condition.isAlwaysTrue()) { + keyExprs = null; + } else if (condition.isAlwaysFalse()) { + keyExprs = null; + } else if (!condition.getNonEquiConditions().isEmpty()) { + throw new IAE("Cannot join lookup with non-equi condition: %s", condition); + } else if (!condition.getEquiConditions() + .stream() + .allMatch(eq -> eq.getRightColumn().equals(LookupColumnSelectorFactory.KEY_COLUMN))) { + throw new IAE("Cannot join lookup with condition referring to non-key column: %s", condition); + } else { + keyExprs = condition.getEquiConditions().stream().map(Equality::getLeftExpr).collect(Collectors.toList()); + } + + return new LookupJoinMatcher(extractor, leftSelectorFactory, condition, keyExprs, remainderNeeded); + } + + @Override + public ColumnSelectorFactory getColumnSelectorFactory() + { + return selectorFactory; + } + + @Override + public void matchCondition() + { + currentIterator = null; + matchingRemainder = false; + + if (condition.isAlwaysFalse()) { + currentEntry.set(null); + } else if (condition.isAlwaysTrue()) { + currentIterator = extractor.iterable().iterator(); + nextMatch(); + } else { + // Not always true, not always false, it's a normal condition. + + if (keySuppliers.isEmpty()) { + currentEntry.set(null); + return; + } + + // In order to match, all keySuppliers must return the same string, which must be a key in the lookup. + String theKey = null; + + for (Supplier keySupplier : keySuppliers) { + final String key = keySupplier.get(); + + if (key == null || (theKey != null && !theKey.equals(key))) { + currentEntry.set(null); + return; + } else { + theKey = key; + } + } + + // All keySuppliers matched. Check if they are actually in the lookup. + final String theValue = extractor.apply(theKey); + + if (theValue != null) { + assert theKey != null; + currentEntry.set(Pair.of(theKey, theValue)); + + if (matchedKeys != null) { + matchedKeys.add(theKey); + } + } else { + currentEntry.set(null); + } + } + } + + @Override + public void matchRemainder() + { + matchingRemainder = true; + + if (condition.isAlwaysFalse()) { + currentIterator = extractor.iterable().iterator(); + } else if (condition.isAlwaysTrue()) { + currentIterator = Collections.emptyIterator(); + } else { + currentIterator = Iterators.filter( + extractor.iterable().iterator(), + entry -> !matchedKeys.contains(entry.getKey()) + ); + } + + nextMatch(); + } + + @Override + public boolean hasMatch() + { + return currentEntry.get() != null; + } + + @Override + public boolean matchingRemainder() + { + return matchingRemainder; + } + + @Override + public void nextMatch() + { + if (currentIterator != null && currentIterator.hasNext()) { + final Map.Entry entry = currentIterator.next(); + currentEntry.set(Pair.of(entry.getKey(), entry.getValue())); + } else { + currentIterator = null; + currentEntry.set(null); + } + } + + @Override + public void reset() + { + // Do not reset matchedKeys; we want to remember it across reset() calls so the 'remainder' is anything + // that was unmatched across _all_ cursor walks. + currentEntry.set(null); + currentIterator = null; + matchingRemainder = false; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java new file mode 100644 index 000000000000..47c9af912e1b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java @@ -0,0 +1,86 @@ +/* + * 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.lookup; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.JoinMatcher; +import org.apache.druid.segment.join.Joinable; + +import javax.annotation.Nullable; +import java.util.List; + +public class LookupJoinable implements Joinable +{ + static final List ALL_COLUMNS = ImmutableList.of( + LookupColumnSelectorFactory.KEY_COLUMN, + LookupColumnSelectorFactory.VALUE_COLUMN + ); + + private final LookupExtractor extractor; + + private LookupJoinable(LookupExtractor extractor) + { + this.extractor = extractor; + } + + public static LookupJoinable wrap(final LookupExtractor extractor) + { + return new LookupJoinable(extractor); + } + + @Override + public List getAvailableColumns() + { + return ALL_COLUMNS; + } + + @Override + public int getCardinality(String columnName) + { + return CARDINALITY_UNKNOWN; + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String columnName) + { + if (ALL_COLUMNS.contains(columnName)) { + return new ColumnCapabilitiesImpl().setType(ValueType.STRING); + } else { + return null; + } + } + + @Override + public JoinMatcher makeJoinMatcher( + final ColumnSelectorFactory leftSelectorFactory, + final JoinConditionAnalysis condition, + final boolean remainderNeeded + ) + { + return LookupJoinMatcher.create(extractor, leftSelectorFactory, condition, remainderNeeded); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java new file mode 100644 index 000000000000..ecfbe61aba20 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java @@ -0,0 +1,53 @@ +/* + * 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 org.apache.druid.segment.column.ValueType; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; + +public interface IndexedTable +{ + List keyColumns(); + + List allColumns(); + + Map rowSignature(); + + int numRows(); + + Index columnIndex(int column); + + Reader columnReader(int column); + + interface Index + { + IntList find(Object key); + } + + interface Reader + { + @Nullable + Object read(int row); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnSelectorFactory.java new file mode 100644 index 000000000000..1cacb630e55d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnSelectorFactory.java @@ -0,0 +1,104 @@ +/* + * 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 org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.NilColumnValueSelector; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ValueType; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.function.IntSupplier; + +public class IndexedTableColumnSelectorFactory implements ColumnSelectorFactory +{ + private final IndexedTable table; + private final IntSupplier currentRow; + + IndexedTableColumnSelectorFactory(IndexedTable table, IntSupplier currentRow) + { + this.table = table; + this.currentRow = currentRow; + } + + @Nullable + static ColumnCapabilities columnCapabilities(final IndexedTable table, final String columnName) + { + final ValueType valueType = table.rowSignature().get(columnName); + + if (valueType != null) { + final ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl().setType(valueType); + + if (valueType == ValueType.STRING) { + // IndexedTables are not _really_ dictionary-encoded, but we fake it using the row number as the dict. code. + capabilities.setDictionaryEncoded(true); + } + + return capabilities; + } else { + return null; + } + } + + @Nonnull + @Override + public DimensionSelector makeDimensionSelector(final DimensionSpec dimensionSpec) + { + final int columnNumber = table.allColumns().indexOf(dimensionSpec.getDimension()); + + if (columnNumber < 0) { + return dimensionSpec.decorate(DimensionSelector.constant(null, dimensionSpec.getExtractionFn())); + } else { + final DimensionSelector undecoratedSelector = new IndexedTableDimensionSelector( + table, + currentRow, + columnNumber, + dimensionSpec.getExtractionFn() + ); + + return dimensionSpec.decorate(undecoratedSelector); + } + } + + @Nonnull + @Override + public ColumnValueSelector makeColumnValueSelector(final String columnName) + { + final int columnNumber = table.allColumns().indexOf(columnName); + + if (columnNumber < 0) { + return NilColumnValueSelector.instance(); + } else { + return new IndexedTableColumnValueSelector(table, currentRow, columnNumber); + } + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(final String column) + { + return columnCapabilities(table, column); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnValueSelector.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnValueSelector.java new file mode 100644 index 000000000000..814658cd0a29 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnValueSelector.java @@ -0,0 +1,132 @@ +/* + * 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 org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; +import java.util.function.IntSupplier; + +public class IndexedTableColumnValueSelector implements ColumnValueSelector +{ + private final IntSupplier currentRow; + private final IndexedTable.Reader columnReader; + + IndexedTableColumnValueSelector(IndexedTable table, IntSupplier currentRow, int columnNumber) + { + this.currentRow = currentRow; + this.columnReader = table.columnReader(columnNumber); + } + + @Override + public double getDouble() + { + final int rowNum = currentRow.getAsInt(); + + if (rowNum != -1) { + final Object value = columnReader.read(currentRow.getAsInt()); + + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } + } + + // Otherwise this shouldn't have been called (due to isNull returning true). + assert NullHandling.replaceWithDefault(); + return NullHandling.defaultDoubleValue(); + } + + @Override + public float getFloat() + { + final int rowNum = currentRow.getAsInt(); + + if (rowNum != -1) { + final Object value = columnReader.read(currentRow.getAsInt()); + + if (value instanceof Number) { + return ((Number) value).floatValue(); + } + } + + // Otherwise this shouldn't have been called (due to isNull returning true). + assert NullHandling.replaceWithDefault(); + return NullHandling.defaultFloatValue(); + } + + @Override + public long getLong() + { + final int rowNum = currentRow.getAsInt(); + + if (rowNum != -1) { + final Object value = columnReader.read(currentRow.getAsInt()); + + if (value instanceof Number) { + return ((Number) value).longValue(); + } + } + + // Otherwise this shouldn't have been called (due to isNull returning true). + assert NullHandling.replaceWithDefault(); + return NullHandling.defaultLongValue(); + } + + @Override + public boolean isNull() + { + final int rowNum = currentRow.getAsInt(); + + if (rowNum == -1) { + return true; + } + + final Object value = columnReader.read(rowNum); + return !(value instanceof Number); + } + + @Nullable + @Override + public Object getObject() + { + final int rowNum = currentRow.getAsInt(); + + if (rowNum == -1) { + return null; + } else { + return columnReader.read(currentRow.getAsInt()); + } + } + + @Override + public Class classOfObject() + { + return Object.class; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("columnReader", columnReader); + inspector.visit("currentRow", currentRow); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableDimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableDimensionSelector.java new file mode 100644 index 000000000000..18936a3e8e1a --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableDimensionSelector.java @@ -0,0 +1,144 @@ +/* + * 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.Predicate; +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.DimensionSelectorUtils; +import org.apache.druid.segment.IdLookup; +import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.data.SingleIndexedInt; + +import javax.annotation.Nullable; +import java.util.function.IntSupplier; + +public class IndexedTableDimensionSelector implements DimensionSelector +{ + private final IndexedTable table; + private final IntSupplier currentRow; + @Nullable + private final ExtractionFn extractionFn; + private final IndexedTable.Reader columnReader; + private final SingleIndexedInt currentIndexedInts; + + IndexedTableDimensionSelector( + IndexedTable table, + IntSupplier currentRow, + int columnNumber, + @Nullable ExtractionFn extractionFn + ) + { + this.table = table; + this.currentRow = currentRow; + this.extractionFn = extractionFn; + this.columnReader = table.columnReader(columnNumber); + this.currentIndexedInts = new SingleIndexedInt(); + } + + @Override + public IndexedInts getRow() + { + final int rowNum = currentRow.getAsInt(); + + if (rowNum == -1) { + // Null value. + currentIndexedInts.setValue(table.numRows()); + } else { + currentIndexedInts.setValue(rowNum); + } + + return currentIndexedInts; + } + + @Override + public ValueMatcher makeValueMatcher(@Nullable String value) + { + return DimensionSelectorUtils.makeValueMatcherGeneric(this, value); + } + + @Override + public ValueMatcher makeValueMatcher(Predicate predicate) + { + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + } + + @Override + public int getValueCardinality() + { + // +1 for nulls. + return table.numRows() + 1; + } + + @Nullable + @Override + public String lookupName(int id) + { + final String value; + + if (id == table.numRows()) { + value = null; + } else { + value = DimensionHandlerUtils.convertObjectToString(columnReader.read(id)); + } + + if (extractionFn == null) { + return value; + } else { + return extractionFn.apply(value); + } + } + + @Override + public boolean nameLookupPossibleInAdvance() + { + return true; + } + + @Nullable + @Override + public IdLookup idLookup() + { + return null; + } + + @Nullable + @Override + public Object getObject() + { + return lookupName(currentRow.getAsInt()); + } + + @Override + public Class classOfObject() + { + return String.class; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("table", table); + inspector.visit("extractionFn", extractionFn); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java new file mode 100644 index 000000000000..5f48ff479f2c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java @@ -0,0 +1,310 @@ +/* + * 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.IntIterator; +import it.unimi.dsi.fastutil.ints.IntIterators; +import it.unimi.dsi.fastutil.ints.IntRBTreeSet; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.BaseDoubleColumnValueSelector; +import org.apache.druid.segment.BaseFloatColumnValueSelector; +import org.apache.druid.segment.BaseLongColumnValueSelector; +import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.ColumnProcessorFactory; +import org.apache.druid.segment.ColumnProcessors; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.join.Equality; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.JoinMatcher; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +public class IndexedTableJoinMatcher implements JoinMatcher +{ + private final IndexedTable table; + private final List> conditionMatchers; + private final IntIterator[] currentMatchedRows; + private final ColumnSelectorFactory selectorFactory; + + // matchedRows and matchingRemainder are used to implement matchRemainder(). + private final IntSet matchedRows; + private boolean matchingRemainder = false; + + // currentIterator and currentRow are used to track iteration position through the currently-matched-rows. + @Nullable + private IntIterator currentIterator; + private int currentRow; + + IndexedTableJoinMatcher( + final IndexedTable table, + final ColumnSelectorFactory leftSelectorFactory, + final JoinConditionAnalysis condition, + final boolean remainderNeeded + ) + { + this.table = table; + + if (condition.isAlwaysTrue()) { + this.conditionMatchers = Collections.singletonList(() -> IntIterators.fromTo(0, table.numRows())); + } else if (condition.isAlwaysFalse()) { + this.conditionMatchers = Collections.singletonList(() -> IntIterators.fromTo(0, 0)); + } else if (condition.getNonEquiConditions().isEmpty()) { + this.conditionMatchers = condition.getEquiConditions() + .stream() + .map(eq -> makeConditionMatcher(table, leftSelectorFactory, eq)) + .collect(Collectors.toList()); + } else { + throw new IAE( + "Cannot build hash-join matcher on non-equi-join condition: %s", + condition.getOriginalExpression() + ); + } + + this.currentMatchedRows = new IntIterator[conditionMatchers.size()]; + this.selectorFactory = new IndexedTableColumnSelectorFactory(table, () -> currentRow); + + if (remainderNeeded) { + this.matchedRows = new IntRBTreeSet(); + } else { + this.matchedRows = null; + } + + } + + private static Supplier makeConditionMatcher( + final IndexedTable table, + final ColumnSelectorFactory selectorFactory, + final Equality condition + ) + { + if (!table.keyColumns().contains(condition.getRightColumn())) { + throw new IAE("Cannot build hash-join matcher on non-key-based condition: %s", condition); + } + + final int keyColumnNumber = table.allColumns().indexOf(condition.getRightColumn()); + final ValueType keyColumnType = table.rowSignature().get(condition.getRightColumn()); + final IndexedTable.Index index = table.columnIndex(keyColumnNumber); + + return ColumnProcessors.makeProcessor( + condition.getLeftExpr(), + keyColumnType, + new ConditionMatcherFactory(keyColumnType, index), + selectorFactory + ); + } + + @Override + public ColumnSelectorFactory getColumnSelectorFactory() + { + return selectorFactory; + } + + @Override + public void matchCondition() + { + reset(); + + for (int i = 0; i < conditionMatchers.size(); i++) { + final IntIterator rows = conditionMatchers.get(i).get(); + if (rows.hasNext()) { + currentMatchedRows[i] = rows; + } else { + return; + } + } + + if (currentMatchedRows.length == 1) { + currentIterator = currentMatchedRows[0]; + } else { + currentIterator = new SortedIntIntersectionIterator(currentMatchedRows); + } + + nextMatch(); + } + + @Override + public void matchRemainder() + { + Preconditions.checkState(matchedRows != null, "matchedRows != null"); + + currentIterator = new IntIterator() + { + int current = -1; + + { + advanceRemainderIterator(); + } + + @Override + public int nextInt() + { + if (current >= table.numRows()) { + throw new NoSuchElementException(); + } + + final int retVal = current; + advanceRemainderIterator(); + return retVal; + } + + @Override + public boolean hasNext() + { + return current < table.numRows(); + } + + private void advanceRemainderIterator() + { + do { + current++; + } while (current < table.numRows() && matchedRows.contains(current)); + } + }; + + matchingRemainder = true; + nextMatch(); + } + + @Override + public boolean matchingRemainder() + { + return matchingRemainder; + } + + @Override + public boolean hasMatch() + { + return currentRow >= 0; + } + + @Override + public void nextMatch() + { + advanceCurrentRow(); + + if (!matchingRemainder && matchedRows != null && hasMatch()) { + matchedRows.add(currentRow); + } + } + + @Override + public void reset() + { + // Do not reset matchedRows; we want to remember it across reset() calls so the 'remainder' is anything + // that was unmatched across _all_ cursor walks. + currentIterator = null; + currentRow = -1; + matchingRemainder = false; + } + + private void advanceCurrentRow() + { + if (currentIterator != null && currentIterator.hasNext()) { + currentRow = currentIterator.nextInt(); + } else { + currentIterator = null; + currentRow = -1; + } + } + + /** + * Makes suppliers that returns the list of IndexedTable rows that match the values from selectors. + */ + private static class ConditionMatcherFactory implements ColumnProcessorFactory> + { + private final ValueType keyType; + private final IndexedTable.Index index; + + ConditionMatcherFactory(ValueType keyType, IndexedTable.Index index) + { + this.keyType = keyType; + this.index = index; + } + + @Override + public ValueType defaultType() + { + return keyType; + } + + @Override + public Supplier makeDimensionProcessor(DimensionSelector selector) + { + return () -> { + final IndexedInts row = selector.getRow(); + + if (row.size() == 1) { + final String key = selector.lookupName(row.get(0)); + return index.find(key).iterator(); + } else { + // Multi-valued rows are not handled by the join system right now; treat them as nulls. + return IntIterators.EMPTY_ITERATOR; + } + }; + } + + @Override + public Supplier makeFloatProcessor(BaseFloatColumnValueSelector selector) + { + if (NullHandling.replaceWithDefault()) { + return () -> index.find(selector.getFloat()).iterator(); + } else { + return () -> selector.isNull() ? IntIterators.EMPTY_ITERATOR : index.find(selector.getFloat()).iterator(); + } + } + + @Override + public Supplier makeDoubleProcessor(BaseDoubleColumnValueSelector selector) + { + if (NullHandling.replaceWithDefault()) { + return () -> index.find(selector.getDouble()).iterator(); + } else { + return () -> selector.isNull() ? IntIterators.EMPTY_ITERATOR : index.find(selector.getDouble()).iterator(); + } + } + + @Override + public Supplier makeLongProcessor(BaseLongColumnValueSelector selector) + { + if (NullHandling.replaceWithDefault()) { + return () -> index.find(selector.getLong()).iterator(); + } else { + return () -> selector.isNull() ? IntIterators.EMPTY_ITERATOR : index.find(selector.getLong()).iterator(); + } + } + + @Override + public Supplier makeComplexProcessor(BaseObjectColumnValueSelector selector) + { + return () -> IntIterators.EMPTY_ITERATOR; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java new file mode 100644 index 000000000000..df46d8869045 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java @@ -0,0 +1,78 @@ +/* + * 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 org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.JoinMatcher; +import org.apache.druid.segment.join.Joinable; + +import javax.annotation.Nullable; +import java.util.List; + +public class IndexedTableJoinable implements Joinable +{ + private final IndexedTable table; + + public IndexedTableJoinable(final IndexedTable table) + { + this.table = table; + } + + @Override + public List getAvailableColumns() + { + return table.allColumns(); + } + + @Override + public int getCardinality(String columnName) + { + if (table.allColumns().contains(columnName)) { + return table.numRows(); + } else { + // NullDimensionSelector has cardinality = 1 (one null, nothing else). + return 1; + } + } + + @Override + @Nullable + public ColumnCapabilities getColumnCapabilities(String columnName) + { + return IndexedTableColumnSelectorFactory.columnCapabilities(table, columnName); + } + + @Override + public JoinMatcher makeJoinMatcher( + final ColumnSelectorFactory leftColumnSelectorFactory, + final JoinConditionAnalysis condition, + final boolean remainderNeeded + ) + { + return new IndexedTableJoinMatcher( + table, + leftColumnSelectorFactory, + condition, + remainderNeeded + ); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexedTable.java new file mode 100644 index 000000000000..dfb5803d00db --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexedTable.java @@ -0,0 +1,166 @@ +/* + * 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 it.unimi.dsi.fastutil.ints.IntLists; +import org.apache.druid.java.util.common.IAE; +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.column.ValueType; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * An IndexedTable composed of a List-based table and Map-based indexes. The implementation is agnostic to the + * specific row type; it uses a {@link RowAdapter} to work with any sort of object. + */ +public class RowBasedIndexedTable implements IndexedTable +{ + private final List table; + private final List> index; + private final Map rowSignature; + private final List columns; + private final List columnTypes; + private final List> columnFunctions; + private final List keyColumns; + + public RowBasedIndexedTable( + final List table, + final RowAdapter rowAdapter, + final Map rowSignature, + final List keyColumns + ) + { + this.table = table; + this.rowSignature = rowSignature; + this.columns = rowSignature.keySet().stream().sorted().collect(Collectors.toList()); + this.columnTypes = new ArrayList<>(columns.size()); + this.columnFunctions = columns.stream().map(rowAdapter::columnFunction).collect(Collectors.toList()); + this.keyColumns = keyColumns; + + if (!rowSignature.keySet().containsAll(keyColumns)) { + throw new ISE( + "keyColumns[%s] must all be contained in rowSignature[%s]", + String.join(", ", keyColumns), + String.join(", ", rowSignature.keySet()) + ); + } + + index = new ArrayList<>(columns.size()); + + for (int i = 0; i < columns.size(); i++) { + final String column = columns.get(i); + final Map m; + final ValueType columnType = rowSignature.get(column); + + columnTypes.add(columnType); + + if (keyColumns.contains(column)) { + final Function columnFunction = columnFunctions.get(i); + + m = new HashMap<>(); + + for (int j = 0; j < table.size(); j++) { + final RowType row = table.get(j); + final Object key = DimensionHandlerUtils.convertObjectToType(columnFunction.apply(row), columnType); + if (key != null) { + final IntList array = m.computeIfAbsent(key, k -> new IntArrayList()); + array.add(j); + } + } + } else { + m = null; + } + + index.add(m); + } + } + + @Override + public List keyColumns() + { + return keyColumns; + } + + @Override + public List allColumns() + { + return columns; + } + + @Override + public Map rowSignature() + { + return rowSignature; + } + + @Override + public Index columnIndex(int column) + { + final Map indexMap = index.get(column); + + if (indexMap == null) { + throw new IAE("Column[%d] is not a key column", column); + } + + final ValueType columnType = columnTypes.get(column); + + 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; + } + }; + } + + @Override + public Reader columnReader(int column) + { + final Function columnFn = columnFunctions.get(column); + + if (columnFn == null) { + throw new IAE("Column[%d] is not a valid column", column); + } + + return row -> columnFn.apply(table.get(row)); + } + + @Override + public int numRows() + { + return table.size(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/SortedIntIntersectionIterator.java b/processing/src/main/java/org/apache/druid/segment/join/table/SortedIntIntersectionIterator.java new file mode 100644 index 000000000000..df09ef586ccd --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/table/SortedIntIntersectionIterator.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.druid.segment.join.table; + +import com.google.common.base.Preconditions; +import it.unimi.dsi.fastutil.ints.IntIterator; + +import java.util.Arrays; +import java.util.NoSuchElementException; + +/** + * Iterates over the intersection of an array of sorted int lists. Intended for situations where the number + * of iterators is fairly small. The iterators must be composed of ascending, nonnegative ints. + * + * @see RowBasedIndexedTable#columnReader uses this + */ +public class SortedIntIntersectionIterator implements IntIterator +{ + private static final int NIL = -1; + + private final IntIterator[] iterators; + private final int[] currents; + + private int next = NIL; + + SortedIntIntersectionIterator(final IntIterator[] iterators) + { + Preconditions.checkArgument(iterators.length > 0, "iterators.length > 0"); + this.iterators = iterators; + this.currents = new int[iterators.length]; + Arrays.fill(currents, NIL); + advance(); + } + + @Override + public int nextInt() + { + if (next == NIL) { + throw new NoSuchElementException(); + } + + final int retVal = next; + + advance(); + + return retVal; + } + + @Override + public boolean hasNext() + { + return next != NIL; + } + + private void advance() + { + next++; + + // This is the part that assumes the number of iterators is fairly small. + boolean foundNext = false; + while (!foundNext) { + foundNext = true; + + for (int i = 0; i < iterators.length; i++) { + while (currents[i] < next && iterators[i].hasNext()) { + currents[i] = iterators[i].nextInt(); + } + + if (currents[i] < next && !iterators[i].hasNext()) { + next = NIL; + return; + } else if (currents[i] > next) { + next = currents[i]; + foundNext = false; + } + } + } + + assert Arrays.stream(currents).allMatch(x -> x == next); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java index 4c90f0e32c26..62bc71f96f99 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java @@ -25,7 +25,7 @@ import org.apache.druid.data.input.Rows; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.query.filter.ValueMatcher; -import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory; +import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.column.ColumnHolder; import org.joda.time.DateTime; diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java index 91fcb6c4f121..56dd6d00c641 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java @@ -210,14 +210,13 @@ public static ColumnValueSelector makeExprEvalSelector( public static DimensionSelector makeDimensionSelector( final ColumnSelectorFactory columnSelectorFactory, final Expr expression, - final ExtractionFn extractionFn + @Nullable final ExtractionFn extractionFn ) { final Expr.BindingDetails exprDetails = expression.analyzeInputs(); Parser.validateExpr(expression, exprDetails); final List columns = exprDetails.getRequiredBindingsList(); - if (columns.size() == 1) { final String column = Iterables.getOnlyElement(columns); final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(column); diff --git a/processing/src/test/java/org/apache/druid/query/extraction/MapLookupExtractorTest.java b/processing/src/test/java/org/apache/druid/query/extraction/MapLookupExtractorTest.java index 0fb777ad3c52..80ffdc2e0603 100644 --- a/processing/src/test/java/org/apache/druid/query/extraction/MapLookupExtractorTest.java +++ b/processing/src/test/java/org/apache/druid/query/extraction/MapLookupExtractorTest.java @@ -19,10 +19,12 @@ package org.apache.druid.query.extraction; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import org.apache.druid.common.config.NullHandling; import org.junit.Assert; +import org.junit.BeforeClass; import org.junit.Test; import java.util.Arrays; @@ -33,9 +35,16 @@ public class MapLookupExtractorTest { - private final Map lookupMap = ImmutableMap.of("foo", "bar", "null", "", "empty String", "", "", "empty_string"); + private final Map lookupMap = + ImmutableMap.of("foo", "bar", "null", "", "empty String", "", "", "empty_string"); private final MapLookupExtractor fn = new MapLookupExtractor(lookupMap, false); + @BeforeClass + public static void setUpClass() + { + NullHandling.initializeForTests(); + } + @Test public void testUnApply() { @@ -81,6 +90,21 @@ public void testGetCacheKey() Assert.assertFalse(Arrays.equals(fn.getCacheKey(), fn4.getCacheKey())); } + @Test + public void testCanIterate() + { + Assert.assertTrue(fn.canIterate()); + } + + @Test + public void testIterable() + { + Assert.assertEquals( + ImmutableList.copyOf(lookupMap.entrySet()), + ImmutableList.copyOf(fn.iterable()) + ); + } + @Test public void testEquals() { diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java index a342071d0c70..b23c70e96fb6 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java @@ -269,12 +269,6 @@ public DateTime getMaxTime() } // stubs below this line not important for tests - @Override - public String getSegmentIdentifier() - { - return null; - } - @Override public Indexed getAvailableDimensions() diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index cdfe8644b1ef..b0011fbcddba 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -45,7 +45,6 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; -import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; @@ -54,6 +53,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ValueType; diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java new file mode 100644 index 000000000000..27de9aaa7cae --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java @@ -0,0 +1,1390 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.filter.ExpressionDimFilter; +import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.join.lookup.LookupJoinable; +import org.apache.druid.segment.join.table.IndexedTable; +import org.apache.druid.segment.join.table.IndexedTableJoinable; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.timeline.SegmentId; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.util.Collections; + +public class HashJoinSegmentStorageAdapterTest +{ + private static final String FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX = "c1."; + private static final String FACT_TO_COUNTRY_ON_NUMBER_PREFIX = "c2."; + private static final String FACT_TO_REGION_PREFIX = "r1."; + private static final String REGION_TO_COUNTRY_PREFIX = "rtc."; + private static Long NULL_COUNTRY; + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + public QueryableIndexSegment factSegment; + public LookupExtractor countryIsoCodeToNameLookup; + public LookupExtractor countryNumberToNameLookup; + public IndexedTable countriesTable; + public IndexedTable regionsTable; + + @BeforeClass + public static void setUpStatic() + { + NullHandling.initializeForTests(); + NULL_COUNTRY = NullHandling.sqlCompatible() ? null : 0L; + } + + @Before + public void setUp() throws IOException + { + factSegment = new QueryableIndexSegment( + JoinTestHelper.createFactIndexBuilder(temporaryFolder.newFolder()).buildMMappedIndex(), + SegmentId.dummy("facts") + ); + countryIsoCodeToNameLookup = JoinTestHelper.createCountryIsoCodeToNameLookup(); + countryNumberToNameLookup = JoinTestHelper.createCountryNumberToNameLookup(); + countriesTable = JoinTestHelper.createCountriesIndexedTable(); + regionsTable = JoinTestHelper.createRegionsIndexedTable(); + } + + @After + public void tearDown() + { + if (factSegment != null) { + factSegment.close(); + } + } + + @Test + public void test_getInterval_factToCountry() + { + Assert.assertEquals( + Intervals.of("2015-09-12/2015-09-12T02:33:40.060Z"), + makeFactToCountrySegment().getInterval() + ); + } + + @Test + public void test_getAvailableDimensions_factToCountry() + { + Assert.assertEquals( + ImmutableList.of( + "channel", + "regionIsoCode", + "countryNumber", + "countryIsoCode", + "user", + "isRobot", + "isAnonymous", + "namespace", + "page", + "delta", + "c1.countryIsoCode", + "c1.countryName", + "c1.countryNumber" + ), + Lists.newArrayList(makeFactToCountrySegment().getAvailableDimensions().iterator()) + ); + } + + @Test + public void test_getAvailableMetrics_factToCountry() + { + Assert.assertEquals( + ImmutableList.of("channel_uniques"), + Lists.newArrayList(makeFactToCountrySegment().getAvailableMetrics().iterator()) + ); + } + + @Test + public void test_getDimensionCardinality_factToCountryFactColumn() + { + Assert.assertEquals( + 15, + makeFactToCountrySegment().getDimensionCardinality("countryIsoCode") + ); + } + + @Test + public void test_getDimensionCardinality_factToCountryJoinColumn() + { + Assert.assertEquals( + 15, + makeFactToCountrySegment().getDimensionCardinality(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName") + ); + } + + @Test + public void test_getDimensionCardinality_factToCountryNonexistentFactColumn() + { + Assert.assertEquals( + 1, + makeFactToCountrySegment().getDimensionCardinality("nonexistent") + ); + } + + @Test + public void test_getDimensionCardinality_factToCountryNonexistentJoinColumn() + { + Assert.assertEquals( + 1, + makeFactToCountrySegment().getDimensionCardinality(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent") + ); + } + + @Test + public void test_getMinTime_factToCountry() + { + Assert.assertEquals( + DateTimes.of("2015-09-12T00:46:58.771Z"), + makeFactToCountrySegment().getMinTime() + ); + } + + @Test + public void test_getMaxTime_factToCountry() + { + Assert.assertEquals( + DateTimes.of("2015-09-12T02:33:40.059Z"), + makeFactToCountrySegment().getMaxTime() + ); + } + + @Test + public void test_getMinValue_factToCountryFactColumn() + { + Assert.assertNull(makeFactToCountrySegment().getMinValue("countryIsoCode")); + } + + @Test + public void test_getMinValue_factToCountryJoinColumn() + { + Assert.assertNull(makeFactToCountrySegment().getMinValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode")); + } + + @Test + public void test_getMinValue_factToCountryNonexistentFactColumn() + { + Assert.assertNull(makeFactToCountrySegment().getMinValue("nonexistent")); + } + + @Test + public void test_getMinValue_factToCountryNonexistentJoinColumn() + { + Assert.assertNull(makeFactToCountrySegment().getMinValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent")); + } + + @Test + public void test_getMaxValue_factToCountryFactColumn() + { + Assert.assertEquals( + "US", + makeFactToCountrySegment().getMaxValue("countryIsoCode") + ); + } + + @Test + public void test_getMaxValue_factToCountryJoinColumn() + { + Assert.assertNull(makeFactToCountrySegment().getMaxValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName")); + } + + @Test + public void test_getMaxValue_factToCountryNonexistentFactColumn() + { + Assert.assertNull(makeFactToCountrySegment().getMaxValue("nonexistent")); + } + + @Test + public void test_getMaxValue_factToCountryNonexistentJoinColumn() + { + Assert.assertNull(makeFactToCountrySegment().getMaxValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent")); + } + + @Test + public void test_getCapabilities_factToCountry() + { + Assert.assertFalse(makeFactToCountrySegment().getCapabilities().dimensionValuesSorted()); + } + + @Test + public void test_getColumnCapabilities_factToCountryFactColumn() + { + final ColumnCapabilities capabilities = makeFactToCountrySegment().getColumnCapabilities("countryIsoCode"); + + Assert.assertEquals(ValueType.STRING, capabilities.getType()); + Assert.assertTrue(capabilities.hasBitmapIndexes()); + Assert.assertTrue(capabilities.isDictionaryEncoded()); + } + + @Test + public void test_getColumnCapabilities_factToCountryJoinColumn() + { + final ColumnCapabilities capabilities = makeFactToCountrySegment().getColumnCapabilities( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode" + ); + + Assert.assertEquals(ValueType.STRING, capabilities.getType()); + Assert.assertFalse(capabilities.hasBitmapIndexes()); + Assert.assertTrue(capabilities.isDictionaryEncoded()); + } + + @Test + public void test_getColumnCapabilities_factToCountryNonexistentFactColumn() + { + final ColumnCapabilities capabilities = makeFactToCountrySegment() + .getColumnCapabilities("nonexistent"); + + Assert.assertNull(capabilities); + } + + @Test + public void test_getColumnCapabilities_factToCountryNonexistentJoinColumn() + { + final ColumnCapabilities capabilities = makeFactToCountrySegment() + .getColumnCapabilities(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent"); + + Assert.assertNull(capabilities); + } + + @Test + public void test_getColumnTypeName_factToCountryFactColumn() + { + Assert.assertEquals("hyperUnique", makeFactToCountrySegment().getColumnTypeName("channel_uniques")); + } + + @Test + public void test_getColumnTypeName_factToCountryJoinColumn() + { + Assert.assertEquals( + "STRING", + makeFactToCountrySegment().getColumnTypeName(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName") + ); + } + + @Test + public void test_getColumnTypeName_factToCountryNonexistentFactColumn() + { + Assert.assertNull(makeFactToCountrySegment().getColumnTypeName("nonexistent")); + } + + @Test + public void test_getColumnTypeName_factToCountryNonexistentJoinColumn() + { + Assert.assertNull( + makeFactToCountrySegment().getColumnTypeName(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent") + ); + } + + @Test + public void test_getMaxIngestedEventTime_factToCountry() + { + Assert.assertEquals( + DateTimes.of("2015-09-12T02:33:40.059Z"), + makeFactToCountrySegment().getMaxIngestedEventTime() + ); + } + + @Test + public void test_getNumRows_factToCountry() + { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Cannot retrieve number of rows from join segment"); + + makeFactToCountrySegment().getNumRows(); + } + + @Test + public void test_getMetadata_factToCountry() + { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Cannot retrieve metadata from join segment"); + + makeFactToCountrySegment().getMetadata(); + } + + @Test + public void test_makeCursors_factToCountryLeft() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)) + ).makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" + ), + ImmutableList.of( + new Object[]{"Talk:Oswald Tilghman", null, null, null, NULL_COUNTRY}, + new Object[]{"Rallicula", null, null, null, NULL_COUNTRY}, + new Object[]{"Peremptory norm", "AU", "AU", "Australia", 0L}, + new Object[]{"Apamea abruzzorum", null, null, null, NULL_COUNTRY}, + new Object[]{"Atractus flammigerus", null, null, null, NULL_COUNTRY}, + new Object[]{"Agama mossambica", null, null, null, NULL_COUNTRY}, + new Object[]{"Mathis Bolly", "MX", "MX", "Mexico", 10L}, + new Object[]{"유희왕 GX", "KR", "KR", "Republic of Korea", 9L}, + new Object[]{"青野武", "JP", "JP", "Japan", 8L}, + new Object[]{"Golpe de Estado en Chile de 1973", "CL", "CL", "Chile", 2L}, + new Object[]{"President of India", "US", "US", "United States", 13L}, + new Object[]{"Diskussion:Sebastian Schulz", "DE", "DE", "Germany", 3L}, + new Object[]{"Saison 9 de Secret Story", "FR", "FR", "France", 5L}, + new Object[]{"Glasgow", "GB", "GB", "United Kingdom", 6L}, + new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, + new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L}, + new Object[]{"Otjiwarongo Airport", "US", "US", "United States", 13L}, + new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L}, + new Object[]{"DirecTV", "US", "US", "United States", 13L}, + new Object[]{"Carlo Curti", "US", "US", "United States", 13L}, + new Object[]{"Giusy Ferreri discography", "IT", "IT", "Italy", 7L}, + new Object[]{"Roma-Bangkok", "IT", "IT", "Italy", 7L}, + new Object[]{"Wendigo", "SV", "SV", "El Salvador", 12L}, + new Object[]{"Алиса в Зазеркалье", "NO", "NO", "Norway", 11L}, + new Object[]{"Gabinete Ministerial de Rafael Correa", "EC", "EC", "Ecuador", 4L}, + new Object[]{"Old Anatolian Turkish", "US", "US", "United States", 13L} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryInner() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryOnIsoCode(JoinType.INNER)) + ).makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" + ), + ImmutableList.of( + new Object[]{"Peremptory norm", "AU", "AU", "Australia", 0L}, + new Object[]{"Mathis Bolly", "MX", "MX", "Mexico", 10L}, + new Object[]{"유희왕 GX", "KR", "KR", "Republic of Korea", 9L}, + new Object[]{"青野武", "JP", "JP", "Japan", 8L}, + new Object[]{"Golpe de Estado en Chile de 1973", "CL", "CL", "Chile", 2L}, + new Object[]{"President of India", "US", "US", "United States", 13L}, + new Object[]{"Diskussion:Sebastian Schulz", "DE", "DE", "Germany", 3L}, + new Object[]{"Saison 9 de Secret Story", "FR", "FR", "France", 5L}, + new Object[]{"Glasgow", "GB", "GB", "United Kingdom", 6L}, + new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, + new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L}, + new Object[]{"Otjiwarongo Airport", "US", "US", "United States", 13L}, + new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L}, + new Object[]{"DirecTV", "US", "US", "United States", 13L}, + new Object[]{"Carlo Curti", "US", "US", "United States", 13L}, + new Object[]{"Giusy Ferreri discography", "IT", "IT", "Italy", 7L}, + new Object[]{"Roma-Bangkok", "IT", "IT", "Italy", 7L}, + new Object[]{"Wendigo", "SV", "SV", "El Salvador", 12L}, + new Object[]{"Алиса в Зазеркалье", "NO", "NO", "Norway", 11L}, + new Object[]{"Gabinete Ministerial de Rafael Correa", "EC", "EC", "Ecuador", 4L}, + new Object[]{"Old Anatolian Turkish", "US", "US", "United States", 13L} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryInnerUsingLookup() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.INNER)) + ).makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "k", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" + ), + ImmutableList.of( + new Object[]{"Peremptory norm", "AU", "AU", "Australia"}, + new Object[]{"Mathis Bolly", "MX", "MX", "Mexico"}, + new Object[]{"유희왕 GX", "KR", "KR", "Republic of Korea"}, + new Object[]{"青野武", "JP", "JP", "Japan"}, + new Object[]{"Golpe de Estado en Chile de 1973", "CL", "CL", "Chile"}, + new Object[]{"President of India", "US", "US", "United States"}, + new Object[]{"Diskussion:Sebastian Schulz", "DE", "DE", "Germany"}, + new Object[]{"Saison 9 de Secret Story", "FR", "FR", "France"}, + new Object[]{"Glasgow", "GB", "GB", "United Kingdom"}, + new Object[]{"Didier Leclair", "CA", "CA", "Canada"}, + new Object[]{"Les Argonautes", "CA", "CA", "Canada"}, + new Object[]{"Otjiwarongo Airport", "US", "US", "United States"}, + new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada"}, + new Object[]{"DirecTV", "US", "US", "United States"}, + new Object[]{"Carlo Curti", "US", "US", "United States"}, + new Object[]{"Giusy Ferreri discography", "IT", "IT", "Italy"}, + new Object[]{"Roma-Bangkok", "IT", "IT", "Italy"}, + new Object[]{"Wendigo", "SV", "SV", "El Salvador"}, + new Object[]{"Алиса в Зазеркалье", "NO", "NO", "Norway"}, + new Object[]{"Gabinete Ministerial de Rafael Correa", "EC", "EC", "Ecuador"}, + new Object[]{"Old Anatolian Turkish", "US", "US", "United States"} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryInnerUsingCountryNumber() + { + // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" + // is interpreted as 0 (a.k.a. Australia). + + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryOnNumber(JoinType.INNER)) + ).makeCursors( + new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", + FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryNumber" + ), + NullHandling.sqlCompatible() ? + ImmutableList.of( + new Object[]{"Peremptory norm", "AU", "AU", "Australia", 0L}, + new Object[]{"President of India", "US", "US", "United States", 13L}, + new Object[]{"Glasgow", "GB", "GB", "United Kingdom", 6L}, + new Object[]{"Otjiwarongo Airport", "US", "US", "United States", 13L}, + new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L}, + new Object[]{"DirecTV", "US", "US", "United States", 13L}, + new Object[]{"Carlo Curti", "US", "US", "United States", 13L}, + new Object[]{"Giusy Ferreri discography", "IT", "IT", "Italy", 7L}, + new Object[]{"Roma-Bangkok", "IT", "IT", "Italy", 7L}, + new Object[]{"Old Anatolian Turkish", "US", "US", "United States", 13L} + ) : + ImmutableList.of( + new Object[]{"Talk:Oswald Tilghman", null, "AU", "Australia", 0L}, + new Object[]{"Peremptory norm", "AU", "AU", "Australia", 0L}, + new Object[]{"President of India", "US", "US", "United States", 13L}, + new Object[]{"Glasgow", "GB", "GB", "United Kingdom", 6L}, + new Object[]{"Otjiwarongo Airport", "US", "US", "United States", 13L}, + new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L}, + new Object[]{"DirecTV", "US", "US", "United States", 13L}, + new Object[]{"Carlo Curti", "US", "US", "United States", 13L}, + new Object[]{"Giusy Ferreri discography", "IT", "IT", "Italy", 7L}, + new Object[]{"Roma-Bangkok", "IT", "IT", "Italy", 7L}, + new Object[]{"Old Anatolian Turkish", "US", "US", "United States", 13L} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryInnerUsingCountryNumberUsingLookup() + { + // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" + // is interpreted as 0 (a.k.a. Australia). + + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryNameUsingNumberLookup(JoinType.INNER)) + ).makeCursors( + new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v" + ), + NullHandling.sqlCompatible() ? + ImmutableList.of( + new Object[]{"Peremptory norm", "AU", "Australia"}, + new Object[]{"President of India", "US", "United States"}, + new Object[]{"Glasgow", "GB", "United Kingdom"}, + new Object[]{"Otjiwarongo Airport", "US", "United States"}, + new Object[]{"Sarah Michelle Gellar", "CA", "Canada"}, + new Object[]{"DirecTV", "US", "United States"}, + new Object[]{"Carlo Curti", "US", "United States"}, + new Object[]{"Giusy Ferreri discography", "IT", "Italy"}, + new Object[]{"Roma-Bangkok", "IT", "Italy"}, + new Object[]{"Old Anatolian Turkish", "US", "United States"} + ) : + ImmutableList.of( + new Object[]{"Talk:Oswald Tilghman", null, "Australia"}, + new Object[]{"Peremptory norm", "AU", "Australia"}, + new Object[]{"President of India", "US", "United States"}, + new Object[]{"Glasgow", "GB", "United Kingdom"}, + new Object[]{"Otjiwarongo Airport", "US", "United States"}, + new Object[]{"Sarah Michelle Gellar", "CA", "Canada"}, + new Object[]{"DirecTV", "US", "United States"}, + new Object[]{"Carlo Curti", "US", "United States"}, + new Object[]{"Giusy Ferreri discography", "IT", "Italy"}, + new Object[]{"Roma-Bangkok", "IT", "Italy"}, + new Object[]{"Old Anatolian Turkish", "US", "United States"} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryLeftWithFilterOnFacts() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)) + ).makeCursors( + new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" + ), + ImmutableList.of( + new Object[]{"Diskussion:Sebastian Schulz", "DE", "DE", "Germany", 3L} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNull() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)) + ).makeCursors( + new SelectorDimFilter("channel", null, null).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + "countryNumber", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" + ), + ImmutableList.of( + new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNull() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)) + ).makeCursors( + new SelectorDimFilter("channel", null, null).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + "countryNumber", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" + ), + ImmutableList.of( + new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryRightWithFilterOnJoinable() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)) + ).makeCursors( + new SelectorDimFilter( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + "Germany", + null + ).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + "countryNumber", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" + ), + ImmutableList.of( + new Object[]{"Diskussion:Sebastian Schulz", "DE", 3L, "DE", "Germany", 3L} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)) + ).makeCursors( + new OrDimFilter( + new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", "DE", null), + new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Norway", null), + new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber", "10", null) + ).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" + ), + ImmutableList.of( + new Object[]{"Mathis Bolly", "MX", "MX", "Mexico", 10L}, + new Object[]{"Diskussion:Sebastian Schulz", "DE", "DE", "Germany", 3L}, + new Object[]{"Алиса в Зазеркалье", "NO", "NO", "Norway", 11L} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryLeftWithFilterOnJoinableUsingLookup() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)) + ).makeCursors( + new OrDimFilter( + new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "k", "DE", null), + new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Norway", null) + ).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "k", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" + ), + ImmutableList.of( + new Object[]{"Diskussion:Sebastian Schulz", "DE", "DE", "Germany"}, + new Object[]{"Алиса в Зазеркалье", "NO", "NO", "Norway"} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondition() + { + // Join condition => always true. + // Filter => Fact to countries on countryIsoCode. + + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.INNER, + JoinConditionAnalysis.forExpression( + "1", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ) + ).makeCursors( + new ExpressionDimFilter( + StringUtils.format("\"%scountryIsoCode\" == countryIsoCode", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), + ExprMacroTable.nil() + ).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" + ), + ImmutableList.of( + new Object[]{"Peremptory norm", "AU", "AU", "Australia", 0L}, + new Object[]{"Mathis Bolly", "MX", "MX", "Mexico", 10L}, + new Object[]{"유희왕 GX", "KR", "KR", "Republic of Korea", 9L}, + new Object[]{"青野武", "JP", "JP", "Japan", 8L}, + new Object[]{"Golpe de Estado en Chile de 1973", "CL", "CL", "Chile", 2L}, + new Object[]{"President of India", "US", "US", "United States", 13L}, + new Object[]{"Diskussion:Sebastian Schulz", "DE", "DE", "Germany", 3L}, + new Object[]{"Saison 9 de Secret Story", "FR", "FR", "France", 5L}, + new Object[]{"Glasgow", "GB", "GB", "United Kingdom", 6L}, + new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, + new Object[]{"Les Argonautes", "CA", "CA", "Canada", 1L}, + new Object[]{"Otjiwarongo Airport", "US", "US", "United States", 13L}, + new Object[]{"Sarah Michelle Gellar", "CA", "CA", "Canada", 1L}, + new Object[]{"DirecTV", "US", "US", "United States", 13L}, + new Object[]{"Carlo Curti", "US", "US", "United States", 13L}, + new Object[]{"Giusy Ferreri discography", "IT", "IT", "Italy", 7L}, + new Object[]{"Roma-Bangkok", "IT", "IT", "Italy", 7L}, + new Object[]{"Wendigo", "SV", "SV", "El Salvador", 12L}, + new Object[]{"Алиса в Зазеркалье", "NO", "NO", "Norway", 11L}, + new Object[]{"Gabinete Ministerial de Rafael Correa", "EC", "EC", "Ecuador", 4L}, + new Object[]{"Old Anatolian Turkish", "US", "US", "United States", 13L} + ) + ); + } + + @Test + public void test_makeCursors_factToRegionToCountryLeft() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) + ) + ).makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Talk:Oswald Tilghman", null, null}, + new Object[]{"Rallicula", null, null}, + new Object[]{"Peremptory norm", "New South Wales", "Australia"}, + new Object[]{"Apamea abruzzorum", null, null}, + new Object[]{"Atractus flammigerus", null, null}, + new Object[]{"Agama mossambica", null, null}, + new Object[]{"Mathis Bolly", "Mexico City", "Mexico"}, + new Object[]{"유희왕 GX", "Seoul", "Republic of Korea"}, + new Object[]{"青野武", "Tōkyō", "Japan"}, + new Object[]{"Golpe de Estado en Chile de 1973", "Santiago Metropolitan", "Chile"}, + new Object[]{"President of India", "California", "United States"}, + new Object[]{"Diskussion:Sebastian Schulz", "Hesse", "Germany"}, + new Object[]{"Saison 9 de Secret Story", "Val d'Oise", "France"}, + new Object[]{"Glasgow", "Kingston upon Hull", "United Kingdom"}, + new Object[]{"Didier Leclair", "Ontario", "Canada"}, + new Object[]{"Les Argonautes", "Quebec", "Canada"}, + new Object[]{"Otjiwarongo Airport", "California", "United States"}, + new Object[]{"Sarah Michelle Gellar", "Ontario", "Canada"}, + new Object[]{"DirecTV", "North Carolina", "United States"}, + new Object[]{"Carlo Curti", "California", "United States"}, + new Object[]{"Giusy Ferreri discography", "Provincia di Varese", "Italy"}, + new Object[]{"Roma-Bangkok", "Provincia di Varese", "Italy"}, + new Object[]{"Wendigo", "Departamento de San Salvador", "El Salvador"}, + new Object[]{"Алиса в Зазеркалье", "Finnmark Fylke", "Norway"}, + new Object[]{"Gabinete Ministerial de Rafael Correa", "Provincia del Guayas", "Ecuador"}, + new Object[]{"Old Anatolian Turkish", "Virginia", "United States"} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryAlwaysTrue() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + "1", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ) + ).makeCursors( + new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, + new Object[]{"Diskussion:Sebastian Schulz", "Canada"}, + new Object[]{"Diskussion:Sebastian Schulz", "Chile"}, + new Object[]{"Diskussion:Sebastian Schulz", "Germany"}, + new Object[]{"Diskussion:Sebastian Schulz", "Ecuador"}, + new Object[]{"Diskussion:Sebastian Schulz", "France"}, + new Object[]{"Diskussion:Sebastian Schulz", "United Kingdom"}, + new Object[]{"Diskussion:Sebastian Schulz", "Italy"}, + new Object[]{"Diskussion:Sebastian Schulz", "Japan"}, + new Object[]{"Diskussion:Sebastian Schulz", "Republic of Korea"}, + new Object[]{"Diskussion:Sebastian Schulz", "Mexico"}, + new Object[]{"Diskussion:Sebastian Schulz", "Norway"}, + new Object[]{"Diskussion:Sebastian Schulz", "El Salvador"}, + new Object[]{"Diskussion:Sebastian Schulz", "United States"}, + new Object[]{"Diskussion:Sebastian Schulz", "Atlantis"} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryAlwaysFalse() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + "0", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ) + ).makeCursors( + new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Diskussion:Sebastian Schulz", null} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + LookupJoinable.wrap(countryIsoCodeToNameLookup), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + "1", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ) + ).makeCursors( + new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" + ), + ImmutableList.of( + new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, + new Object[]{"Diskussion:Sebastian Schulz", "Canada"}, + new Object[]{"Diskussion:Sebastian Schulz", "Chile"}, + new Object[]{"Diskussion:Sebastian Schulz", "Germany"}, + new Object[]{"Diskussion:Sebastian Schulz", "Ecuador"}, + new Object[]{"Diskussion:Sebastian Schulz", "France"}, + new Object[]{"Diskussion:Sebastian Schulz", "United Kingdom"}, + new Object[]{"Diskussion:Sebastian Schulz", "Italy"}, + new Object[]{"Diskussion:Sebastian Schulz", "Japan"}, + new Object[]{"Diskussion:Sebastian Schulz", "Republic of Korea"}, + new Object[]{"Diskussion:Sebastian Schulz", "Mexico"}, + new Object[]{"Diskussion:Sebastian Schulz", "Norway"}, + new Object[]{"Diskussion:Sebastian Schulz", "El Salvador"}, + new Object[]{"Diskussion:Sebastian Schulz", "United States"}, + new Object[]{"Diskussion:Sebastian Schulz", "Atlantis"} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + LookupJoinable.wrap(countryIsoCodeToNameLookup), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + "0", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ) + ).makeCursors( + new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" + ), + ImmutableList.of( + new Object[]{"Diskussion:Sebastian Schulz", null} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryUsingVirtualColumn() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.INNER, + JoinConditionAnalysis.forExpression( + StringUtils.format("\"%scountryIsoCode\" == virtual", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ) + ).makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.create( + Collections.singletonList( + new ExpressionVirtualColumn( + "virtual", + "concat(substring(countryIsoCode, 0, 1),'L')", + ValueType.STRING, + ExprMacroTable.nil() + ) + ) + ), + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + "virtual", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Golpe de Estado en Chile de 1973", "CL", "CL", "CL", "Chile"}, + new Object[]{"Didier Leclair", "CA", "CL", "CL", "Chile"}, + new Object[]{"Les Argonautes", "CA", "CL", "CL", "Chile"}, + new Object[]{"Sarah Michelle Gellar", "CA", "CL", "CL", "Chile"} + ) + ); + } + + @Test + public void test_makeCursors_factToCountryUsingExpression() + { + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.INNER, + JoinConditionAnalysis.forExpression( + StringUtils.format( + "\"%scountryIsoCode\" == concat(substring(countryIsoCode, 0, 1),'L')", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + ), + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ) + ).makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Golpe de Estado en Chile de 1973", "CL", "CL", "Chile"}, + new Object[]{"Didier Leclair", "CA", "CL", "Chile"}, + new Object[]{"Les Argonautes", "CA", "CL", "Chile"}, + new Object[]{"Sarah Michelle Gellar", "CA", "CL", "Chile"} + ) + ); + } + + @Test + public void test_makeCursors_factToRegionTheWrongWay() + { + // Joins using only regionIsoCode, which is wrong since they are not unique internationally. + + JoinTestHelper.verifyCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of( + new JoinableClause( + FACT_TO_REGION_PREFIX, + new IndexedTableJoinable(regionsTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + StringUtils.format( + "\"%sregionIsoCode\" == regionIsoCode", + FACT_TO_REGION_PREFIX + ), + FACT_TO_REGION_PREFIX, + ExprMacroTable.nil() + ) + ) + ) + ).makeCursors( + new SelectorDimFilter("regionIsoCode", "VA", null).toFilter(), + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "regionIsoCode", + "countryIsoCode", + FACT_TO_REGION_PREFIX + "regionName", + FACT_TO_REGION_PREFIX + "countryIsoCode" + ), + ImmutableList.of( + new Object[]{"Giusy Ferreri discography", "VA", "IT", "Provincia di Varese", "IT"}, + new Object[]{"Giusy Ferreri discography", "VA", "IT", "Virginia", "US"}, + new Object[]{"Roma-Bangkok", "VA", "IT", "Provincia di Varese", "IT"}, + new Object[]{"Roma-Bangkok", "VA", "IT", "Virginia", "US"}, + new Object[]{"Old Anatolian Turkish", "VA", "US", "Provincia di Varese", "IT"}, + new Object[]{"Old Anatolian Turkish", "VA", "US", "Virginia", "US"} + ) + ); + } + + @Test + public void test_makeCursors_errorOnNonEquiJoin() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Cannot build hash-join matcher on non-equi-join condition: x == y"); + + JoinTestHelper.readCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + "x == y", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ) + ).makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of() + ); + } + + @Test + public void test_makeCursors_errorOnNonKeyBasedJoin() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Cannot build hash-join matcher on non-key-based condition: " + + "Equality{leftExpr=x, rightColumn='countryName'}"); + + JoinTestHelper.readCursors( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + StringUtils.format("x == \"%scountryName\"", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ) + ) + ).makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of() + ); + } + + private JoinableClause factToCountryNameUsingIsoCodeLookup(final JoinType joinType) + { + return new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + LookupJoinable.wrap(countryIsoCodeToNameLookup), + joinType, + JoinConditionAnalysis.forExpression( + StringUtils.format("\"%sk\" == countryIsoCode", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ); + } + + private JoinableClause factToCountryNameUsingNumberLookup(final JoinType joinType) + { + return new JoinableClause( + FACT_TO_COUNTRY_ON_NUMBER_PREFIX, + LookupJoinable.wrap(countryNumberToNameLookup), + joinType, + JoinConditionAnalysis.forExpression( + StringUtils.format("\"%sk\" == countryNumber", FACT_TO_COUNTRY_ON_NUMBER_PREFIX), + FACT_TO_COUNTRY_ON_NUMBER_PREFIX, + ExprMacroTable.nil() + ) + ); + } + + private JoinableClause factToCountryOnIsoCode(final JoinType joinType) + { + return new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + joinType, + JoinConditionAnalysis.forExpression( + StringUtils.format("\"%scountryIsoCode\" == countryIsoCode", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + ExprMacroTable.nil() + ) + ); + } + + private JoinableClause factToCountryOnNumber(final JoinType joinType) + { + return new JoinableClause( + FACT_TO_COUNTRY_ON_NUMBER_PREFIX, + new IndexedTableJoinable(countriesTable), + joinType, + JoinConditionAnalysis.forExpression( + StringUtils.format("\"%scountryNumber\" == countryNumber", FACT_TO_COUNTRY_ON_NUMBER_PREFIX), + FACT_TO_COUNTRY_ON_NUMBER_PREFIX, + ExprMacroTable.nil() + ) + ); + } + + private JoinableClause factToRegion(final JoinType joinType) + { + return new JoinableClause( + FACT_TO_REGION_PREFIX, + new IndexedTableJoinable(regionsTable), + joinType, + JoinConditionAnalysis.forExpression( + StringUtils.format( + "\"%sregionIsoCode\" == regionIsoCode && \"%scountryIsoCode\" == countryIsoCode", + FACT_TO_REGION_PREFIX, + FACT_TO_REGION_PREFIX + ), + FACT_TO_REGION_PREFIX, + ExprMacroTable.nil() + ) + ); + } + + private JoinableClause regionToCountry(final JoinType joinType) + { + return new JoinableClause( + REGION_TO_COUNTRY_PREFIX, + new IndexedTableJoinable(countriesTable), + joinType, + JoinConditionAnalysis.forExpression( + StringUtils.format( + "\"%scountryIsoCode\" == \"%scountryIsoCode\"", + FACT_TO_REGION_PREFIX, + REGION_TO_COUNTRY_PREFIX + ), + REGION_TO_COUNTRY_PREFIX, + ExprMacroTable.nil() + ) + ); + } + + private HashJoinSegmentStorageAdapter makeFactToCountrySegment() + { + return new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)) + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java new file mode 100644 index 000000000000..f0ce5c82e7e0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java @@ -0,0 +1,138 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.join.table.IndexedTableJoinable; +import org.apache.druid.timeline.SegmentId; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; + +public class HashJoinSegmentTest +{ + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private QueryableIndexSegment baseSegment; + private HashJoinSegment hashJoinSegmentNoClauses; + private HashJoinSegment hashJoinSegmentManyClauses; + + @BeforeClass + public static void setUpStatic() + { + NullHandling.initializeForTests(); + } + + @Before + public void setUp() throws IOException + { + baseSegment = new QueryableIndexSegment( + JoinTestHelper.createFactIndexBuilder(temporaryFolder.newFolder()).buildMMappedIndex(), + SegmentId.dummy("facts") + ); + + hashJoinSegmentNoClauses = new HashJoinSegment( + baseSegment, + ImmutableList.of() + ); + + hashJoinSegmentManyClauses = new HashJoinSegment( + baseSegment, + ImmutableList.of( + new JoinableClause( + "j0.", + new IndexedTableJoinable(JoinTestHelper.createCountriesIndexedTable()), + JoinType.LEFT, + JoinConditionAnalysis.forExpression("1", "j0.", ExprMacroTable.nil()) + ), + new JoinableClause( + "j1.", + new IndexedTableJoinable(JoinTestHelper.createRegionsIndexedTable()), + JoinType.LEFT, + JoinConditionAnalysis.forExpression("1", "j1.", ExprMacroTable.nil()) + ) + ) + ); + } + + @Test + public void test_getId_noClauses() + { + Assert.assertEquals(baseSegment.getId(), hashJoinSegmentNoClauses.getId()); + } + + @Test + public void test_getId_manyClauses() + { + Assert.assertEquals(baseSegment.getId(), hashJoinSegmentManyClauses.getId()); + } + + @Test + public void test_getDataInterval_noClauses() + { + Assert.assertEquals(baseSegment.getDataInterval(), hashJoinSegmentNoClauses.getDataInterval()); + } + + @Test + public void test_getDataInterval_manyClauses() + { + Assert.assertEquals(baseSegment.getDataInterval(), hashJoinSegmentManyClauses.getDataInterval()); + } + + @Test + public void test_asQueryableIndex_noClauses() + { + Assert.assertNull(hashJoinSegmentNoClauses.asQueryableIndex()); + } + + @Test + public void test_asQueryableIndex_manyClauses() + { + Assert.assertNull(hashJoinSegmentManyClauses.asQueryableIndex()); + } + + @Test + public void test_asStorageAdapter_noClauses() + { + Assert.assertThat( + hashJoinSegmentNoClauses.asStorageAdapter(), + CoreMatchers.instanceOf(HashJoinSegmentStorageAdapter.class) + ); + } + + @Test + public void test_asStorageAdapter_manyClauses() + { + Assert.assertThat( + hashJoinSegmentManyClauses.asStorageAdapter(), + CoreMatchers.instanceOf(HashJoinSegmentStorageAdapter.class) + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinConditionAnalysisTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinConditionAnalysisTest.java new file mode 100644 index 000000000000..f5ec8b8062cc --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinConditionAnalysisTest.java @@ -0,0 +1,293 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.math.expr.Expr; +import org.apache.druid.math.expr.ExprMacroTable; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.List; +import java.util.stream.Collectors; + +public class JoinConditionAnalysisTest +{ + private static final String PREFIX = "j."; + + @BeforeClass + public static void setUpStatic() + { + NullHandling.initializeForTests(); + } + + @Test + public void test_forExpression_simple() + { + final String expression = "x == \"j.y\""; + final JoinConditionAnalysis analysis = analyze(expression); + + Assert.assertEquals(expression, analysis.getOriginalExpression()); + Assert.assertTrue(analysis.canHashJoin()); + Assert.assertFalse(analysis.isAlwaysTrue()); + Assert.assertFalse(analysis.isAlwaysFalse()); + Assert.assertEquals( + ImmutableList.of(Pair.of("x", "y")), + equalitiesToPairs(analysis.getEquiConditions()) + ); + Assert.assertEquals( + ImmutableList.of(), + exprsToStrings(analysis.getNonEquiConditions()) + ); + } + + @Test + public void test_forExpression_simpleFlipped() + { + final String expression = "\"j.y\" == x"; + final JoinConditionAnalysis analysis = analyze(expression); + + Assert.assertEquals(expression, analysis.getOriginalExpression()); + Assert.assertTrue(analysis.canHashJoin()); + Assert.assertFalse(analysis.isAlwaysTrue()); + Assert.assertFalse(analysis.isAlwaysFalse()); + Assert.assertEquals( + ImmutableList.of(Pair.of("x", "y")), + equalitiesToPairs(analysis.getEquiConditions()) + ); + Assert.assertEquals( + ImmutableList.of(), + exprsToStrings(analysis.getNonEquiConditions()) + ); + } + + @Test + public void test_forExpression_leftFunction() + { + final String expression = "x + y == \"j.z\""; + final JoinConditionAnalysis analysis = analyze(expression); + + Assert.assertEquals(expression, analysis.getOriginalExpression()); + Assert.assertTrue(analysis.canHashJoin()); + Assert.assertFalse(analysis.isAlwaysTrue()); + Assert.assertFalse(analysis.isAlwaysFalse()); + Assert.assertEquals( + ImmutableList.of(Pair.of("(+ x y)", "z")), + equalitiesToPairs(analysis.getEquiConditions()) + ); + Assert.assertEquals( + ImmutableList.of(), + exprsToStrings(analysis.getNonEquiConditions()) + ); + } + + @Test + public void test_forExpression_rightFunction() + { + final String expression = "\"j.x\" + \"j.y\" == z"; + final JoinConditionAnalysis analysis = analyze(expression); + + Assert.assertEquals(expression, analysis.getOriginalExpression()); + Assert.assertFalse(analysis.canHashJoin()); + Assert.assertFalse(analysis.isAlwaysTrue()); + Assert.assertFalse(analysis.isAlwaysFalse()); + Assert.assertEquals( + ImmutableList.of(), + equalitiesToPairs(analysis.getEquiConditions()) + ); + Assert.assertEquals( + ImmutableList.of("(== (+ j.x j.y) z)"), + exprsToStrings(analysis.getNonEquiConditions()) + ); + } + + @Test + public void test_forExpression_mixedFunction() + { + final String expression = "x + \"j.y\" == \"j.z\""; + final JoinConditionAnalysis analysis = analyze(expression); + + Assert.assertEquals(expression, analysis.getOriginalExpression()); + Assert.assertFalse(analysis.canHashJoin()); + Assert.assertFalse(analysis.isAlwaysTrue()); + Assert.assertFalse(analysis.isAlwaysFalse()); + Assert.assertEquals( + ImmutableList.of(), + equalitiesToPairs(analysis.getEquiConditions()) + ); + Assert.assertEquals( + ImmutableList.of("(== (+ x j.y) j.z)"), + exprsToStrings(analysis.getNonEquiConditions()) + ); + } + + @Test + public void test_forExpression_trueConstant() + { + final String expression = "1 + 1"; + final JoinConditionAnalysis analysis = analyze(expression); + + Assert.assertEquals(expression, analysis.getOriginalExpression()); + Assert.assertTrue(analysis.canHashJoin()); + Assert.assertTrue(analysis.isAlwaysTrue()); + Assert.assertFalse(analysis.isAlwaysFalse()); + Assert.assertEquals( + ImmutableList.of(), + equalitiesToPairs(analysis.getEquiConditions()) + ); + Assert.assertEquals( + ImmutableList.of("2"), + exprsToStrings(analysis.getNonEquiConditions()) + ); + } + + @Test + public void test_forExpression_falseConstant() + { + final String expression = "0"; + final JoinConditionAnalysis analysis = analyze(expression); + + Assert.assertEquals(expression, analysis.getOriginalExpression()); + Assert.assertTrue(analysis.canHashJoin()); + Assert.assertFalse(analysis.isAlwaysTrue()); + Assert.assertTrue(analysis.isAlwaysFalse()); + Assert.assertEquals( + ImmutableList.of(), + equalitiesToPairs(analysis.getEquiConditions()) + ); + Assert.assertEquals( + ImmutableList.of("0"), + exprsToStrings(analysis.getNonEquiConditions()) + ); + } + + @Test + public void test_forExpression_onlyLeft() + { + final String expression = "x == 1"; + final JoinConditionAnalysis analysis = analyze(expression); + + Assert.assertEquals(expression, analysis.getOriginalExpression()); + Assert.assertFalse(analysis.canHashJoin()); + Assert.assertFalse(analysis.isAlwaysTrue()); + Assert.assertFalse(analysis.isAlwaysFalse()); + Assert.assertEquals( + ImmutableList.of(), + equalitiesToPairs(analysis.getEquiConditions()) + ); + Assert.assertEquals( + ImmutableList.of("(== x 1)"), + exprsToStrings(analysis.getNonEquiConditions()) + ); + } + + @Test + public void test_forExpression_onlyRight() + { + final String expression = "\"j.x\" == 1"; + final JoinConditionAnalysis analysis = analyze(expression); + + Assert.assertEquals(expression, analysis.getOriginalExpression()); + Assert.assertTrue(analysis.canHashJoin()); + Assert.assertFalse(analysis.isAlwaysTrue()); + Assert.assertFalse(analysis.isAlwaysFalse()); + Assert.assertEquals( + ImmutableList.of(Pair.of("1", "x")), + equalitiesToPairs(analysis.getEquiConditions()) + ); + Assert.assertEquals( + ImmutableList.of(), + exprsToStrings(analysis.getNonEquiConditions()) + ); + } + + @Test + public void test_forExpression_andOfThreeConditions() + { + final String expression = "(x == \"j.y\") && (x + y == \"j.z\") && (z == \"j.zz\")"; + final JoinConditionAnalysis analysis = analyze(expression); + + Assert.assertEquals(expression, analysis.getOriginalExpression()); + Assert.assertTrue(analysis.canHashJoin()); + Assert.assertFalse(analysis.isAlwaysTrue()); + Assert.assertFalse(analysis.isAlwaysFalse()); + Assert.assertEquals( + ImmutableList.of(Pair.of("x", "y"), Pair.of("(+ x y)", "z"), Pair.of("z", "zz")), + equalitiesToPairs(analysis.getEquiConditions()) + ); + Assert.assertEquals( + ImmutableList.of(), + exprsToStrings(analysis.getNonEquiConditions()) + ); + } + + @Test + public void test_forExpression_mixedAndWithOr() + { + final String expression = "(x == \"j.y\") && ((x + y == \"j.z\") || (z == \"j.zz\"))"; + final JoinConditionAnalysis analysis = analyze(expression); + + Assert.assertEquals(expression, analysis.getOriginalExpression()); + Assert.assertFalse(analysis.canHashJoin()); + Assert.assertFalse(analysis.isAlwaysTrue()); + Assert.assertFalse(analysis.isAlwaysFalse()); + Assert.assertEquals( + ImmutableList.of(Pair.of("x", "y")), + equalitiesToPairs(analysis.getEquiConditions()) + ); + Assert.assertEquals( + ImmutableList.of("(|| (== (+ x y) j.z) (== z j.zz))"), + exprsToStrings(analysis.getNonEquiConditions()) + ); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(JoinConditionAnalysis.class) + .usingGetClass() + .withIgnoredFields("equiConditions", "nonEquiConditions") + .verify(); + } + + private static JoinConditionAnalysis analyze(final String expression) + { + return JoinConditionAnalysis.forExpression(expression, PREFIX, ExprMacroTable.nil()); + } + + private static Pair equalityToPair(final Equality equality) + { + return Pair.of(equality.getLeftExpr().toString(), equality.getRightColumn()); + } + + private static List> equalitiesToPairs(final List equalities) + { + return equalities.stream().map(JoinConditionAnalysisTest::equalityToPair).collect(Collectors.toList()); + } + + private static List exprsToStrings(final List exprs) + { + return exprs.stream().map(String::valueOf).collect(Collectors.toList()); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java new file mode 100644 index 000000000000..88d36e61e2c4 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -0,0 +1,351 @@ +/* + * 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; + +import com.fasterxml.jackson.databind.MappingIterator; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +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.Sequences; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import org.apache.druid.query.extraction.MapLookupExtractor; +import org.apache.druid.segment.BaseDoubleColumnValueSelector; +import org.apache.druid.segment.BaseFloatColumnValueSelector; +import org.apache.druid.segment.BaseLongColumnValueSelector; +import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.ColumnProcessorFactory; +import org.apache.druid.segment.ColumnProcessors; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.RowAdapter; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.join.table.RowBasedIndexedTable; +import org.junit.Assert; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.function.ToLongFunction; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class JoinTestHelper +{ + private static final Logger log = new Logger(JoinTestHelper.class); + private static final String FACT_TIME_COLUMN = "time"; + private static final List FACT_DIMENSIONS = ImmutableList.of( + new StringDimensionSchema("channel"), + new StringDimensionSchema("regionIsoCode"), + new LongDimensionSchema("countryNumber"), + new StringDimensionSchema("countryIsoCode"), + new StringDimensionSchema("user"), + new StringDimensionSchema("isRobot"), + new StringDimensionSchema("isAnonymous"), + new StringDimensionSchema("namespace"), + new StringDimensionSchema("page"), + new LongDimensionSchema("delta") + ); + private static final Map COUNTRIES_SIGNATURE = + ImmutableMap.builder() + .put("countryNumber", ValueType.LONG) + .put("countryIsoCode", ValueType.STRING) + .put("countryName", ValueType.STRING) + .build(); + private static final Map REGIONS_SIGNATURE = + ImmutableMap.builder() + .put("regionIsoCode", ValueType.STRING) + .put("countryIsoCode", ValueType.STRING) + .put("regionName", ValueType.STRING) + .build(); + + private static final ColumnProcessorFactory> SIMPLE_READER = + new ColumnProcessorFactory>() + { + @Override + public ValueType defaultType() + { + return ValueType.STRING; + } + + @Override + public Supplier makeDimensionProcessor(DimensionSelector selector) + { + return selector::defaultGetObject; + } + + @Override + public Supplier makeFloatProcessor(BaseFloatColumnValueSelector selector) + { + return () -> NullHandling.sqlCompatible() && selector.isNull() ? null : selector.getFloat(); + } + + @Override + public Supplier makeDoubleProcessor(BaseDoubleColumnValueSelector selector) + { + return () -> NullHandling.sqlCompatible() && selector.isNull() ? null : selector.getDouble(); + } + + @Override + public Supplier makeLongProcessor(BaseLongColumnValueSelector selector) + { + return () -> NullHandling.sqlCompatible() && selector.isNull() ? null : selector.getLong(); + } + + @Override + public Supplier makeComplexProcessor(BaseObjectColumnValueSelector selector) + { + return selector::getObject; + } + }; + + private static RowAdapter> createMapRowAdapter(final Map signature) + { + return new RowAdapter>() + { + @Override + public ToLongFunction> timestampFunction() + { + return row -> 0L; + } + + @Override + public Function, Object> columnFunction(String columnName) + { + final ValueType columnType = signature.get(columnName); + + if (columnType == null) { + return row -> row.get(columnName); + } else { + return row -> DimensionHandlerUtils.convertObjectToType(row.get(columnName), columnType, false); + } + } + }; + } + + public static IndexBuilder createFactIndexBuilder(final File tmpDir) throws IOException + { + return createFactIndexBuilder(tmpDir, -1); + } + + public static IndexBuilder createFactIndexBuilder(final File tmpDir, final int numRows) throws IOException + { + return withRowsFromResource( + "/wikipedia/data.json", + rows -> IndexBuilder + .create() + .tmpDir(tmpDir) + .schema( + new IncrementalIndexSchema.Builder() + .withDimensionsSpec(new DimensionsSpec(FACT_DIMENSIONS)) + .withMetrics(new HyperUniquesAggregatorFactory("channel_uniques", "channel")) + .withQueryGranularity(Granularities.NONE) + .withRollup(false) + .withMinTimestamp(DateTimes.of("2015-09-12").getMillis()) + .build() + ) + .rows( + () -> + IntStream.rangeClosed(0, numRows < 0 ? 0 : (numRows / rows.size() + 1)) + .boxed() + .flatMap( + i -> + rows.stream() + .map( + row -> + (InputRow) new MapBasedInputRow( + DateTimes.of((String) row.get(FACT_TIME_COLUMN)), + FACT_DIMENSIONS.stream() + .map(DimensionSchema::getName) + .collect(Collectors.toList()), + row + ) + ) + ) + .limit(numRows < 0 ? Long.MAX_VALUE : numRows) + .iterator() + ) + ); + } + + public static MapLookupExtractor createCountryIsoCodeToNameLookup() throws IOException + { + return withRowsFromResource( + "/wikipedia/countries.json", + rows -> { + final LinkedHashMap lookupMap = new LinkedHashMap<>(); + + for (Map row : rows) { + lookupMap.put( + (String) row.get("countryIsoCode"), + (String) row.get("countryName") + ); + } + + return new MapLookupExtractor(lookupMap, false); + } + ); + } + + public static MapLookupExtractor createCountryNumberToNameLookup() throws IOException + { + return withRowsFromResource( + "/wikipedia/countries.json", + rows -> new MapLookupExtractor( + rows.stream() + .collect( + Collectors.toMap( + row -> row.get("countryNumber").toString(), + row -> (String) row.get("countryName") + ) + ), + false + ) + ); + } + + public static RowBasedIndexedTable> createCountriesIndexedTable() throws IOException + { + return withRowsFromResource( + "/wikipedia/countries.json", + rows -> new RowBasedIndexedTable<>( + rows, + createMapRowAdapter(COUNTRIES_SIGNATURE), + COUNTRIES_SIGNATURE, + ImmutableList.of("countryNumber", "countryIsoCode") + ) + ); + } + + public static RowBasedIndexedTable> createRegionsIndexedTable() throws IOException + { + return withRowsFromResource( + "/wikipedia/regions.json", + rows -> new RowBasedIndexedTable<>( + rows, + createMapRowAdapter(REGIONS_SIGNATURE), + REGIONS_SIGNATURE, + ImmutableList.of("regionIsoCode", "countryIsoCode") + ) + ); + } + + public static List readCursors(final Sequence cursors, final List columns) + { + return cursors.flatMap( + cursor -> { + final List> readers = columns + .stream() + .map( + column -> + ColumnProcessors.makeProcessor( + column, + SIMPLE_READER, + cursor.getColumnSelectorFactory() + ) + ) + .collect(Collectors.toList()); + + final List rows = new ArrayList<>(); + + while (!cursor.isDone()) { + final Object[] row = new Object[columns.size()]; + + for (int i = 0; i < row.length; i++) { + row[i] = readers.get(i).get(); + } + + rows.add(row); + cursor.advance(); + } + + return Sequences.simple(rows); + } + ).toList(); + } + + public static void verifyCursors( + final Sequence cursors, + final List columns, + final List expectedRows + ) + { + final List rows = readCursors(cursors, columns); + + for (int i = 0; i < rows.size(); i++) { + try { + log.info("Row #%-2d: %s", i, TestHelper.JSON_MAPPER.writeValueAsString(rows.get(i))); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + Assert.assertEquals("number of rows", expectedRows.size(), rows.size()); + + for (int i = 0; i < rows.size(); i++) { + Assert.assertArrayEquals("row #" + i, expectedRows.get(i), rows.get(i)); + } + } + + private static T withRowsFromResource( + final String resource, + final Function>, T> f + ) throws IOException + { + final ObjectMapper jsonMapper = TestHelper.JSON_MAPPER; + + try ( + final InputStream in = JoinTestHelper.class.getResourceAsStream(resource); + final MappingIterator> iter = jsonMapper.readValues( + jsonMapper.getFactory().createParser(in), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ) + ) { + if (in == null) { + throw new ISE("No such resource: %s", resource); + } + return f.apply(Lists.newArrayList(iter)); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinableClauseTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinableClauseTest.java new file mode 100644 index 000000000000..9ea6e9773de3 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinableClauseTest.java @@ -0,0 +1,113 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.segment.join.table.IndexedTableJoinable; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class JoinableClauseTest +{ + public ExpectedException expectedException = ExpectedException.none(); + + private Joinable joinable; + private JoinableClause clause; + + @Before + public void setUp() throws Exception + { + joinable = new IndexedTableJoinable(JoinTestHelper.createCountriesIndexedTable()); + clause = new JoinableClause( + "j.", + joinable, + JoinType.LEFT, + JoinConditionAnalysis.forExpression("\"j.x\" == y", "j.", ExprMacroTable.nil()) + ); + } + + @Test + public void test_getPrefix() + { + Assert.assertEquals("j.", clause.getPrefix()); + } + + @Test + public void test_getJoinable() + { + Assert.assertEquals(joinable, clause.getJoinable()); + } + + @Test + public void test_getJoinType() + { + Assert.assertEquals(JoinType.LEFT, clause.getJoinType()); + } + + @Test + public void test_getCondition() + { + Assert.assertEquals("\"j.x\" == y", clause.getCondition().getOriginalExpression()); + } + + @Test + public void test_getAvailableColumnsPrefixed() + { + Assert.assertEquals( + ImmutableList.of("j.countryIsoCode", "j.countryName", "j.countryNumber"), + clause.getAvailableColumnsPrefixed() + ); + } + + @Test + public void test_includesColumn_included() + { + Assert.assertTrue(clause.includesColumn("j.countryNumber")); + } + + @Test + public void test_includesColumn_notIncluded() + { + Assert.assertFalse(clause.includesColumn("countryNumber")); + } + + @Test + public void test_unprefix_included() + { + Assert.assertEquals("countryNumber", clause.unprefix("j.countryNumber")); + } + + @Test + public void test_unprefix_notIncluded() + { + expectedException.expect(IllegalArgumentException.class); + clause.includesColumn("countryNumber"); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(JoinableClause.class).usingGetClass().verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/join/PossiblyNullDimensionSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PossiblyNullDimensionSelectorTest.java new file mode 100644 index 000000000000..123d1c351658 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/join/PossiblyNullDimensionSelectorTest.java @@ -0,0 +1,143 @@ +/* + * 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; + +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.ints.IntList; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.data.IndexedInts; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +public class PossiblyNullDimensionSelectorTest +{ + private boolean isNull = false; + + private final DimensionSelector onNullSelector = makeSelector(DimensionSelector.constant(null)); + private final DimensionSelector onNonnullSelector = makeSelector(DimensionSelector.constant("foo")); + + @BeforeClass + public static void setUpClass() + { + NullHandling.initializeForTests(); + } + + @Test + public void test_getRow_normalOnNullSelector() + { + isNull = false; + assertRowsEqual(new int[]{0}, onNullSelector.getRow()); + } + + @Test + public void test_getRow_nullOnNullSelector() + { + isNull = true; + assertRowsEqual(new int[]{0}, onNullSelector.getRow()); + } + + @Test + public void test_getRow_normalOnNonnullSelector() + { + isNull = false; + assertRowsEqual(new int[]{1}, onNonnullSelector.getRow()); + } + + @Test + public void test_getRow_nullOnNonnullSelector() + { + isNull = true; + assertRowsEqual(new int[]{0}, onNonnullSelector.getRow()); + } + + @Test + public void test_getValueCardinality_onNullSelector() + { + Assert.assertEquals(1, onNullSelector.getValueCardinality()); + } + + @Test + public void test_getValueCardinality_onNonnullSelector() + { + Assert.assertEquals(2, onNonnullSelector.getValueCardinality()); + } + + @Test + public void test_lookupName_onNullSelector() + { + Assert.assertNull(onNullSelector.lookupName(0)); + } + + @Test + public void test_lookupName_onNonnullSelector() + { + Assert.assertNull(onNonnullSelector.lookupName(0)); + Assert.assertEquals("foo", onNonnullSelector.lookupName(1)); + } + + @Test + public void test_lookupId_onNullSelector() + { + Assert.assertEquals(0, onNullSelector.idLookup().lookupId(null)); + } + + @Test + public void test_lookupId_onNonnullSelector() + { + Assert.assertEquals(0, onNonnullSelector.idLookup().lookupId(null)); + Assert.assertEquals(1, onNonnullSelector.idLookup().lookupId("foo")); + } + + @Test + public void test_nameLookupPossibleInAdvance_onNullSelector() + { + Assert.assertTrue(onNonnullSelector.nameLookupPossibleInAdvance()); + } + + @Test + public void test_nameLookupPossibleInAdvance_onNonnullSelector() + { + Assert.assertTrue(onNonnullSelector.nameLookupPossibleInAdvance()); + } + + private DimensionSelector makeSelector(final DimensionSelector baseSelector) + { + return new PossiblyNullDimensionSelector(baseSelector, () -> isNull); + } + + private static void assertRowsEqual(final int[] expected, final IndexedInts actual) + { + Assert.assertEquals(IntArrayList.wrap(expected), toList(actual)); + } + + private static IntList toList(final IndexedInts ints) + { + final IntList retVal = new IntArrayList(ints.size()); + + final int size = ints.size(); + for (int i = 0; i < size; i++) { + retVal.add(ints.get(i)); + } + + return retVal; + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexedTableTest.java new file mode 100644 index 000000000000..d57f23a0fa4e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexedTableTest.java @@ -0,0 +1,183 @@ +/* + * 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.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.join.JoinTestHelper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.IOException; +import java.util.Map; + +public class RowBasedIndexedTableTest +{ + // Indexes of fields within the "countries" and "regions" tables. + private static final int INDEX_COUNTRIES_COUNTRY_ISO_CODE = 0; + private static final int INDEX_COUNTRIES_COUNTRY_NAME = 1; + private static final int INDEX_COUNTRIES_COUNTRY_NUMBER = 2; + private static final int INDEX_REGIONS_REGION_ISO_CODE = 1; + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + public RowBasedIndexedTable> countriesTable; + public RowBasedIndexedTable> regionsTable; + + @BeforeClass + public static void setUpStatic() + { + NullHandling.initializeForTests(); + } + + @Before + public void setUp() throws IOException + { + countriesTable = JoinTestHelper.createCountriesIndexedTable(); + regionsTable = JoinTestHelper.createRegionsIndexedTable(); + } + + @Test + public void test_keyColumns_countries() + { + Assert.assertEquals(ImmutableList.of("countryNumber", "countryIsoCode"), countriesTable.keyColumns()); + } + + @Test + public void test_allColumns_countries() + { + Assert.assertEquals( + ImmutableList.of("countryIsoCode", "countryName", "countryNumber"), + countriesTable.allColumns() + ); + } + + @Test + public void test_rowSignature_countries() + { + Assert.assertEquals( + ImmutableMap.builder() + .put("countryIsoCode", ValueType.STRING) + .put("countryName", ValueType.STRING) + .put("countryNumber", ValueType.LONG) + .build(), + countriesTable.rowSignature() + ); + } + + @Test + public void test_numRows_countries() + { + Assert.assertEquals(15, countriesTable.numRows()); + } + + @Test + public void test_columnIndex_countriesCountryIsoCode() + { + final IndexedTable.Index index = countriesTable.columnIndex(INDEX_COUNTRIES_COUNTRY_ISO_CODE); + + Assert.assertEquals(ImmutableList.of(), index.find(null)); + Assert.assertEquals(ImmutableList.of(), index.find(2)); + Assert.assertEquals(ImmutableList.of(13), index.find("US")); + } + + @Test + public void test_columnIndex_countriesCountryNumber() + { + final IndexedTable.Index index = countriesTable.columnIndex(INDEX_COUNTRIES_COUNTRY_NUMBER); + + Assert.assertEquals(ImmutableList.of(), index.find(null)); + Assert.assertEquals(ImmutableList.of(0), index.find(0)); + Assert.assertEquals(ImmutableList.of(0), index.find(0.0)); + Assert.assertEquals(ImmutableList.of(0), index.find("0")); + Assert.assertEquals(ImmutableList.of(2), index.find(2)); + Assert.assertEquals(ImmutableList.of(2), index.find(2.0)); + Assert.assertEquals(ImmutableList.of(2), index.find("2")); + Assert.assertEquals(ImmutableList.of(), index.find(20)); + Assert.assertEquals(ImmutableList.of(), index.find("US")); + } + + @Test + public void test_columnIndex_countriesCountryName() + { + expectedException.expectMessage("Column[1] is not a key column"); + countriesTable.columnIndex(INDEX_COUNTRIES_COUNTRY_NAME); + } + + @Test + public void test_columnIndex_countriesOutOfBounds() + { + expectedException.expect(IndexOutOfBoundsException.class); + countriesTable.columnIndex(99); + } + + @Test + public void test_columnIndex_regionsRegionIsoCode() + { + final IndexedTable.Index index = regionsTable.columnIndex(INDEX_REGIONS_REGION_ISO_CODE); + + Assert.assertEquals(ImmutableList.of(), index.find(null)); + Assert.assertEquals(ImmutableList.of(0), index.find("11")); + Assert.assertEquals(ImmutableList.of(1), index.find(13)); + Assert.assertEquals(ImmutableList.of(12), index.find("QC")); + Assert.assertEquals(ImmutableList.of(15, 16), index.find("VA")); + } + + @Test + public void test_columnReader_countriesCountryNumber() + { + final IndexedTable.Reader reader = countriesTable.columnReader(INDEX_COUNTRIES_COUNTRY_NUMBER); + + Assert.assertEquals(0L, reader.read(0)); + Assert.assertEquals(1L, reader.read(1)); + } + + @Test + public void test_columnReader_countriesCountryName() + { + final IndexedTable.Reader reader = countriesTable.columnReader(INDEX_COUNTRIES_COUNTRY_NAME); + + Assert.assertEquals("Australia", reader.read(0)); + Assert.assertEquals("Canada", reader.read(1)); + Assert.assertEquals("Atlantis", reader.read(14)); + } + + @Test + public void test_columnReader_countriesOutOfBoundsRow() + { + final IndexedTable.Reader reader = countriesTable.columnReader(INDEX_COUNTRIES_COUNTRY_NUMBER); + expectedException.expect(IndexOutOfBoundsException.class); + reader.read(99); + } + + @Test + public void test_columnReader_countriesOutOfBoundsColumn() + { + expectedException.expect(IndexOutOfBoundsException.class); + countriesTable.columnReader(99); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/SortedIntIntersectionIteratorTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/SortedIntIntersectionIteratorTest.java new file mode 100644 index 000000000000..c2d5058de5e0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/join/table/SortedIntIntersectionIteratorTest.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.druid.segment.join.table; + +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.ints.IntIterator; +import it.unimi.dsi.fastutil.ints.IntList; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +public class SortedIntIntersectionIteratorTest +{ + @Test + public void test_iterator_allPossibleSingleListsWithCardinalityUpToThree() + { + // 8 possibilities + for (int i = 0; i < 8; i++) { + final IntList ints = intsFromBits(i); + Assert.assertEquals(ints.toString(), ints, intersection(ints)); + } + } + + @Test + public void test_iterator_allPossibleSetsOfTwoListsWithCardinalityUpToSix() + { + // 4096 possibilities: 64 for each list, 2 lists + for (int i = 0; i < 4096; i++) { + final int bits1 = i & 63; + final int bits2 = (i >> 6) & 63; + + final IntList ints1 = intsFromBits(bits1); + final IntList ints2 = intsFromBits(bits2); + + Assert.assertEquals( + StringUtils.format("ints1 = %s; ints2 = %s", ints1, ints2), + intsFromBits(bits1 & bits2), + intersection(ints1, ints2) + ); + } + } + + @Test + public void test_iterator_allPossibleSetsOfThreeListsWithCardinalityUpToFour() + { + // 4096 possibilities: 16 for each list, 3 lists + for (int i = 0; i < 4096; i++) { + final int bits1 = i & 15; + final int bits2 = (i >> 4) & 15; + final int bits3 = (i >> 8) & 15; + + final IntList ints1 = intsFromBits(bits1); + final IntList ints2 = intsFromBits(bits2); + final IntList ints3 = intsFromBits(bits3); + + Assert.assertEquals( + StringUtils.format("ints1 = %s; ints2 = %s; ints3 = %s", ints1, ints2, ints3), + intsFromBits(bits1 & bits2 & bits3), + intersection(ints1, ints2, ints3) + ); + } + } + + private static IntList intersection(final IntList... lists) + { + final SortedIntIntersectionIterator comboIterator = new SortedIntIntersectionIterator( + Arrays.stream(lists) + .map(IntList::iterator) + .toArray(IntIterator[]::new) + ); + + return new IntArrayList(comboIterator); + } + + private static IntList intsFromBits(final int bits) + { + final IntArrayList retVal = new IntArrayList(4); + + for (int i = 0; i < 32; i++) { + if (((bits >> i) & 1) == 1) { + retVal.add(i); + } + } + + return retVal; + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java index 4675852da2b1..82a9d8020e33 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java @@ -35,13 +35,13 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.BucketExtractionFn; import org.apache.druid.query.filter.ValueMatcher; -import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory; import org.apache.druid.segment.BaseFloatColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.column.ValueType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; diff --git a/processing/src/test/resources/wikipedia/countries.json b/processing/src/test/resources/wikipedia/countries.json new file mode 100644 index 000000000000..ac813e60b6c4 --- /dev/null +++ b/processing/src/test/resources/wikipedia/countries.json @@ -0,0 +1,15 @@ +{"countryNumber":0,"countryIsoCode":"AU","countryName":"Australia"} +{"countryNumber":1,"countryIsoCode":"CA","countryName":"Canada"} +{"countryNumber":2,"countryIsoCode":"CL","countryName":"Chile"} +{"countryNumber":3,"countryIsoCode":"DE","countryName":"Germany"} +{"countryNumber":4,"countryIsoCode":"EC","countryName":"Ecuador"} +{"countryNumber":5,"countryIsoCode":"FR","countryName":"France"} +{"countryNumber":6,"countryIsoCode":"GB","countryName":"United Kingdom"} +{"countryNumber":7,"countryIsoCode":"IT","countryName":"Italy"} +{"countryNumber":8,"countryIsoCode":"JP","countryName":"Japan"} +{"countryNumber":9,"countryIsoCode":"KR","countryName":"Republic of Korea"} +{"countryNumber":10,"countryIsoCode":"MX","countryName":"Mexico"} +{"countryNumber":11,"countryIsoCode":"NO","countryName":"Norway"} +{"countryNumber":12,"countryIsoCode":"SV","countryName":"El Salvador"} +{"countryNumber":13,"countryIsoCode":"US","countryName":"United States"} +{"countryNumber":14,"countryIsoCode":"AX","countryName":"Atlantis"} diff --git a/processing/src/test/resources/wikipedia/data.json b/processing/src/test/resources/wikipedia/data.json new file mode 100644 index 000000000000..026e39c1870e --- /dev/null +++ b/processing/src/test/resources/wikipedia/data.json @@ -0,0 +1,26 @@ +{"time":"2015-09-12T00:46:58.771Z","channel":"#en.wikipedia","regionIsoCode":null,"countryNumber":null,"countryIsoCode":null,"user":"GELongstreet","delta":36,"isRobot":false,"isAnonymous":false,"page":"Talk:Oswald Tilghman","namespace":"Talk"} +{"time":"2015-09-12T00:47:00.496Z","channel":"#ca.wikipedia","regionIsoCode":null,"countryNumber":null,"countryIsoCode":null,"user":"PereBot","delta":17,"isRobot":true,"isAnonymous":false,"page":"Rallicula","namespace":"Main"} +{"time":"2015-09-12T00:47:05.474Z","channel":"#en.wikipedia","regionIsoCode":"NSW","countryNumber":0,"countryIsoCode":"AU","user":"60.225.66.142","delta":0,"isRobot":false,"isAnonymous":true,"page":"Peremptory norm","namespace":"Main"} +{"time":"2015-09-12T00:47:08.770Z","channel":"#vi.wikipedia","regionIsoCode":null,"countryNumber":null,"countryIsoCode":null,"user":"Cheers!-bot","delta":18,"isRobot":true,"isAnonymous":false,"page":"Apamea abruzzorum","namespace":"Main"} +{"time":"2015-09-12T00:47:11.862Z","channel":"#vi.wikipedia","regionIsoCode":null,"countryNumber":null,"countryIsoCode":null,"user":"ThitxongkhoiAWB","delta":18,"isRobot":true,"isAnonymous":false,"page":"Atractus flammigerus","namespace":"Main"} +{"time":"2015-09-12T00:47:13.987Z","channel":"#vi.wikipedia","regionIsoCode":null,"countryNumber":null,"countryIsoCode":null,"user":"ThitxongkhoiAWB","delta":18,"isRobot":true,"isAnonymous":false,"page":"Agama mossambica","namespace":"Main"} +{"time":"2015-09-12T00:48:02.596Z","channel":"#es.wikipedia","regionIsoCode":"DIF","countryNumber":10,"countryIsoCode":"MX","user":"189.217.75.123","delta":-67,"isRobot":false,"isAnonymous":true,"page":"Mathis Bolly","namespace":"Main"} +{"time":"2015-09-12T00:48:07.282Z","channel":"#ko.wikipedia","regionIsoCode":"11","countryNumber":9,"countryIsoCode":"KR","user":"110.47.57.14","delta":663,"isRobot":false,"isAnonymous":true,"page":"유희왕 GX","namespace":"Main"} +{"time":"2015-09-12T00:48:10.048Z","channel":"#ja.wikipedia","regionIsoCode":"13","countryNumber":8,"countryIsoCode":"JP","user":"119.26.148.34","delta":0,"isRobot":false,"isAnonymous":true,"page":"青野武","namespace":"Main"} +{"time":"2015-09-12T00:48:15.753Z","channel":"#es.wikipedia","regionIsoCode":"RM","countryNumber":2,"countryIsoCode":"CL","user":"190.208.152.43","delta":161,"isRobot":false,"isAnonymous":true,"page":"Golpe de Estado en Chile de 1973","namespace":"Main"} +{"time":"2015-09-12T00:48:20.157Z","channel":"#en.wikipedia","regionIsoCode":"CA","countryNumber":13,"countryIsoCode":"US","user":"73.162.114.225","delta":-26,"isRobot":false,"isAnonymous":true,"page":"President of India","namespace":"Main"} +{"time":"2015-09-12T00:49:23.066Z","channel":"#de.wikipedia","regionIsoCode":"HE","countryNumber":3,"countryIsoCode":"DE","user":"93.221.150.186","delta":167,"isRobot":false,"isAnonymous":true,"page":"Diskussion:Sebastian Schulz","namespace":"Diskussion"} +{"time":"2015-09-12T00:50:42.345Z","channel":"#fr.wikipedia","regionIsoCode":"95","countryNumber":5,"countryIsoCode":"FR","user":"92.151.101.96","delta":-1,"isRobot":false,"isAnonymous":true,"page":"Saison 9 de Secret Story","namespace":"Main"} +{"time":"2015-09-12T00:51:20.426Z","channel":"#en.wikipedia","regionIsoCode":"KHL","countryNumber":6,"countryIsoCode":"GB","user":"83.100.174.82","delta":-44,"isRobot":false,"isAnonymous":true,"page":"Glasgow","namespace":"Main"} +{"time":"2015-09-12T00:51:39.371Z","channel":"#fr.wikipedia","regionIsoCode":"ON","countryNumber":1,"countryIsoCode":"CA","user":"173.206.10.49","delta":79,"isRobot":false,"isAnonymous":true,"page":"Didier Leclair","namespace":"Main"} +{"time":"2015-09-12T00:51:45.666Z","channel":"#fr.wikipedia","regionIsoCode":"QC","countryNumber":1,"countryIsoCode":"CA","user":"24.122.168.111","delta":-18,"isRobot":false,"isAnonymous":true,"page":"Les Argonautes","namespace":"Main"} +{"time":"2015-09-12T00:55:21.269Z","channel":"#en.wikipedia","regionIsoCode":"CA","countryNumber":13,"countryIsoCode":"US","user":"75.19.32.31","delta":6,"isRobot":false,"isAnonymous":true,"page":"Otjiwarongo Airport","namespace":"Main"} +{"time":"2015-09-12T00:55:34.021Z","channel":"#en.wikipedia","regionIsoCode":"ON","countryNumber":1,"countryIsoCode":"CA","user":"174.95.51.147","delta":-432,"isRobot":false,"isAnonymous":true,"page":"Sarah Michelle Gellar","namespace":"Main"} +{"time":"2015-09-12T00:58:52.482Z","channel":"#en.wikipedia","regionIsoCode":"NC","countryNumber":13,"countryIsoCode":"US","user":"68.95.102.224","delta":-62,"isRobot":false,"isAnonymous":true,"page":"DirecTV","namespace":"Main"} +{"time":"2015-09-12T00:59:15.437Z","channel":"#en.wikipedia","regionIsoCode":"CA","countryNumber":13,"countryIsoCode":"US","user":"75.36.162.245","delta":82,"isRobot":false,"isAnonymous":true,"page":"Carlo Curti","namespace":"Main"} +{"time":"2015-09-12T00:59:53.272Z","channel":"#en.wikipedia","regionIsoCode":"VA","countryNumber":7,"countryIsoCode":"IT","user":"95.237.203.188","delta":0,"isRobot":false,"isAnonymous":true,"page":"Giusy Ferreri discography","namespace":"Main"} +{"time":"2015-09-12T01:00:06.696Z","channel":"#en.wikipedia","regionIsoCode":"VA","countryNumber":7,"countryIsoCode":"IT","user":"95.237.203.188","delta":0,"isRobot":false,"isAnonymous":true,"page":"Roma-Bangkok","namespace":"Main"} +{"time":"2015-09-12T01:00:51.258Z","channel":"#es.wikipedia","regionIsoCode":"SS","countryNumber":12,"countryIsoCode":"SV","user":"190.150.14.176","delta":-1,"isRobot":false,"isAnonymous":true,"page":"Wendigo","namespace":"Main"} +{"time":"2015-09-12T01:01:00.474Z","channel":"#ru.wikipedia","regionIsoCode":"20","countryNumber":11,"countryIsoCode":"NO","user":"85.113.179.226","delta":48,"isRobot":false,"isAnonymous":true,"page":"Алиса в Зазеркалье","namespace":"Main"} +{"time":"2015-09-12T01:02:08.440Z","channel":"#es.wikipedia","regionIsoCode":"G","countryNumber":4,"countryIsoCode":"EC","user":"181.39.132.136","delta":29,"isRobot":false,"isAnonymous":true,"page":"Gabinete Ministerial de Rafael Correa","namespace":"Main"} +{"time":"2015-09-12T02:33:40.059Z","channel":"#en.wikipedia","regionIsoCode":"VA","countryNumber":13,"countryIsoCode":"US","user":"68.100.166.227","delta":14,"isRobot":false,"isAnonymous":true,"page":"Old Anatolian Turkish","namespace":"Main"} diff --git a/processing/src/test/resources/wikipedia/regions.json b/processing/src/test/resources/wikipedia/regions.json new file mode 100644 index 000000000000..2f22bfa9966a --- /dev/null +++ b/processing/src/test/resources/wikipedia/regions.json @@ -0,0 +1,17 @@ +{"regionIsoCode":"11","countryIsoCode":"KR","regionName":"Seoul"} +{"regionIsoCode":"13","countryIsoCode":"JP","regionName":"Tōkyō"} +{"regionIsoCode":"20","countryIsoCode":"NO","regionName":"Finnmark Fylke"} +{"regionIsoCode":"95","countryIsoCode":"FR","regionName":"Val d'Oise"} +{"regionIsoCode":"CA","countryIsoCode":"US","regionName":"California"} +{"regionIsoCode":"DIF","countryIsoCode":"MX","regionName":"Mexico City"} +{"regionIsoCode":"G","countryIsoCode":"EC","regionName":"Provincia del Guayas"} +{"regionIsoCode":"HE","countryIsoCode":"DE","regionName":"Hesse"} +{"regionIsoCode":"KHL","countryIsoCode":"GB","regionName":"Kingston upon Hull"} +{"regionIsoCode":"NC","countryIsoCode":"US","regionName":"North Carolina"} +{"regionIsoCode":"NSW","countryIsoCode":"AU","regionName":"New South Wales"} +{"regionIsoCode":"ON","countryIsoCode":"CA","regionName":"Ontario"} +{"regionIsoCode":"QC","countryIsoCode":"CA","regionName":"Quebec"} +{"regionIsoCode":"RM","countryIsoCode":"CL","regionName":"Santiago Metropolitan"} +{"regionIsoCode":"SS","countryIsoCode":"SV","regionName":"Departamento de San Salvador"} +{"regionIsoCode":"VA","countryIsoCode":"IT","regionName":"Provincia di Varese"} +{"regionIsoCode":"VA","countryIsoCode":"US","regionName":"Virginia"}