diff --git a/api/src/main/java/io/druid/data/input/Row.java b/api/src/main/java/io/druid/data/input/Row.java index b8ed24c803d1..4a314fb76e19 100644 --- a/api/src/main/java/io/druid/data/input/Row.java +++ b/api/src/main/java/io/druid/data/input/Row.java @@ -65,7 +65,7 @@ public interface Row extends Comparable /** * Returns the raw dimension value for the given column name. This is different from {@link #getDimension} which - * all values to strings before returning them. + * converts all values to strings before returning them. * * @param dimension the column name of the dimension requested * diff --git a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java index 31a73c89a7b8..4abdee17878d 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java @@ -19,8 +19,6 @@ package io.druid.benchmark; -// Run FloatCompressionBenchmarkFileGenerator to generate the required files before running this benchmark - import com.google.common.base.Supplier; import com.google.common.io.Files; import io.druid.segment.data.ColumnarFloats; @@ -44,6 +42,9 @@ import java.util.Random; import java.util.concurrent.TimeUnit; +/** + * Run {@link FloatCompressionBenchmarkFileGenerator} to generate the required files before running this benchmark + */ @State(Scope.Benchmark) @Fork(value = 1) @Warmup(iterations = 10) diff --git a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java index 470f25f00128..27fce45956e3 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java @@ -42,8 +42,9 @@ import java.util.Random; import java.util.concurrent.TimeUnit; -// Run LongCompressionBenchmarkFileGenerator to generate the required files before running this benchmark - +/** + * Run {@link LongCompressionBenchmarkFileGenerator} to generate the required files before running this benchmark + */ @State(Scope.Benchmark) @Fork(value = 1) @Warmup(iterations = 10) diff --git a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java index bf5383b5fbf6..46b5e4d01e73 100644 --- a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java @@ -45,7 +45,7 @@ import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexIndexableAdapter; -import io.druid.segment.Rowboat; +import io.druid.segment.RowIterator; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.timeline.DataSegment; @@ -309,11 +309,11 @@ private void verifyJob(IndexGeneratorJob job) throws IOException QueryableIndex index = HadoopDruidIndexerConfig.INDEX_IO.loadIndex(dir); QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); - for (Rowboat row : adapter.getRows()) { - Object[] metrics = row.getMetrics(); - - rowCount++; - Assert.assertTrue(metrics.length == 2); + try (RowIterator rowIt = adapter.getRows()) { + while (rowIt.moveToNext()) { + rowCount++; + Assert.assertEquals(2, rowIt.getPointer().getNumMetrics()); + } } } Assert.assertEquals(rowCount, data.size()); diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java index 0c8f5f2e257b..3551f2c5e543 100644 --- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java @@ -254,8 +254,7 @@ public void buildingSketchesAtIngestionTime() throws Exception double[] histogram = (double[]) histogramObject; Assert.assertEquals(4, histogram.length); for (final double bin : histogram) { - Assert.assertEquals(100, bin, 100 * 0.2); // 400 items uniformly - // distributed into 4 bins + Assert.assertEquals(100, bin, 100 * 0.2); // 400 items uniformly distributed into 4 bins } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java index 4f0d9d4c81a3..f80c4c68a58d 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java @@ -306,14 +306,7 @@ public static final SerializeResult toBytes( } //writing all metrics - Supplier supplier = new Supplier() - { - @Override - public InputRow get() - { - return row; - } - }; + Supplier supplier = () -> row; WritableUtils.writeVInt(out, aggs.length); for (AggregatorFactory aggFactory : aggs) { String k = aggFactory.getName(); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java index c181d2eb2fce..62bdb5624249 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.indexing.common.TaskToolbox; @@ -35,8 +34,8 @@ import io.druid.segment.IndexSpec; import io.druid.segment.IndexableAdapter; import io.druid.segment.QueryableIndexIndexableAdapter; -import io.druid.segment.Rowboat; -import io.druid.segment.RowboatFilteringIndexAdapter; +import io.druid.segment.RowFilteringIndexAdapter; +import io.druid.segment.RowPointer; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; @@ -120,18 +119,9 @@ public SegmentToMergeHolder apply(PartitionChunk chunkInput) List adapters = Lists.newArrayList(); for (final SegmentToMergeHolder holder : segmentsToMerge) { adapters.add( - new RowboatFilteringIndexAdapter( - new QueryableIndexIndexableAdapter( - toolbox.getIndexIO().loadIndex(holder.getFile()) - ), - new Predicate() - { - @Override - public boolean apply(Rowboat input) - { - return holder.getInterval().contains(input.getTimestamp()); - } - } + new RowFilteringIndexAdapter( + new QueryableIndexIndexableAdapter(toolbox.getIndexIO().loadIndex(holder.getFile())), + (RowPointer rowPointer) -> holder.getInterval().contains(rowPointer.getTimestamp()) ) ); } diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java index d5e6790c9ab9..ccd88442c817 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java @@ -70,7 +70,6 @@ public abstract class AggregatorFactory implements Cacheable * @see AggregateCombiner * @see io.druid.segment.IndexMerger */ - @SuppressWarnings("unused") // Going to be used when https://github.com/druid-io/druid/projects/2 is complete public AggregateCombiner makeAggregateCombiner() { throw new UOE("[%s] does not implement makeAggregateCombiner()", this.getClass().getName()); diff --git a/processing/src/main/java/io/druid/query/extraction/ExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/ExtractionFn.java index 9f7afb8b21c4..a0eb1c2061ff 100644 --- a/processing/src/main/java/io/druid/query/extraction/ExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/ExtractionFn.java @@ -28,6 +28,12 @@ import javax.annotation.Nullable; /** + * An ExtractionFn is a function that can be used to transform the values of a column (typically a dimension). + * Note that ExtractionFn implementations are expected to be Threadsafe. + * + * A simple example of the type of operation this enables is the RegexDimExtractionFn which applies a + * regular expression with a capture group. When the regular expression matches the value of a dimension, + * the value captured by the group is used for grouping operations instead of the dimension value. */ @ExtensionPoint @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @@ -48,14 +54,6 @@ @JsonSubTypes.Type(name = "bucket", value = BucketExtractionFn.class), @JsonSubTypes.Type(name = "strlen", value = StrlenExtractionFn.class) }) -/** - * An ExtractionFn is a function that can be used to transform the values of a column (typically a dimension). - * Note that ExtractionFn implementations are expected to be Threadsafe. - * - * A simple example of the type of operation this enables is the RegexDimExtractionFn which applies a - * regular expression with a capture group. When the regular expression matches the value of a dimension, - * the value captured by the group is used for grouping operations instead of the dimension value. - */ public interface ExtractionFn extends Cacheable { /** diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index d6104f3018e3..ade15e96cf69 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -211,6 +211,7 @@ public VirtualColumns getVirtualColumns() return virtualColumns; } + @Nullable @JsonProperty("filter") public DimFilter getDimFilter() { diff --git a/processing/src/main/java/io/druid/segment/ColumnValueSelector.java b/processing/src/main/java/io/druid/segment/ColumnValueSelector.java index c6a147d16c02..890eab11f7d2 100644 --- a/processing/src/main/java/io/druid/segment/ColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/ColumnValueSelector.java @@ -37,4 +37,5 @@ public interface ColumnValueSelector extends BaseLongColumnValueSelector, BaseDoubleColumnValueSelector, BaseFloatColumnValueSelector, BaseObjectColumnValueSelector { + ColumnValueSelector[] EMPTY_ARRAY = new ColumnValueSelector[0]; } diff --git a/processing/src/main/java/io/druid/segment/Cursor.java b/processing/src/main/java/io/druid/segment/Cursor.java index 3300f5bbc527..3f55baa3c095 100644 --- a/processing/src/main/java/io/druid/segment/Cursor.java +++ b/processing/src/main/java/io/druid/segment/Cursor.java @@ -22,8 +22,15 @@ import org.joda.time.DateTime; /** + * Cursor is an interface for iteration over a range of data points, used during query execution. {@link + * QueryableIndexStorageAdapter.QueryableIndexCursor} is an implementation for historical segments, and {@link + * io.druid.segment.incremental.IncrementalIndexStorageAdapter.IncrementalIndexCursor} is an implementation for {@link + * io.druid.segment.incremental.IncrementalIndex}. + * + * Cursor is conceptually similar to {@link TimeAndDimsPointer}, but the latter is used for historical segment creation + * rather than query execution (as Cursor). If those abstractions could be collapsed (and if it is worthwhile) is yet to + * be determined. */ - public interface Cursor { ColumnSelectorFactory getColumnSelectorFactory(); diff --git a/processing/src/main/java/io/druid/segment/DimensionHandler.java b/processing/src/main/java/io/druid/segment/DimensionHandler.java index d80b86be7cea..bc6f0f869665 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandler.java @@ -20,12 +20,11 @@ package io.druid.segment; import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; -import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.data.Indexed; +import io.druid.segment.selector.settable.SettableColumnValueSelector; import io.druid.segment.writeout.SegmentWriteOutMedium; -import java.io.Closeable; +import java.util.Comparator; /** * Processing related interface @@ -40,10 +39,8 @@ * * This interface allows type-specific behavior column logic, such as choice of indexing structures and disk formats. * to be contained within a type-specific set of handler objects, simplifying processing classes - * such as IncrementalIndex and IndexMerger and allowing for abstracted development of additional dimension types. - * - * A dimension may have two representations, an encoded representation and a actual representation. - * For example, a value for a String dimension has an integer dictionary encoding, and an actual String representation. + * such as {@link io.druid.segment.incremental.IncrementalIndex} and {@link IndexMerger} and allowing for abstracted + * development of additional dimension types. * * A DimensionHandler is a stateless object, and thus thread-safe; its methods should be pure functions. * @@ -85,7 +82,6 @@ default MultiValueHandling getMultivalueHandling() */ DimensionIndexer makeIndexer(); - /** * Creates a new DimensionMergerV9, a per-dimension object responsible for merging indexes/row data across segments * and building the on-disk representation of a dimension. For use with IndexMergerV9 only. @@ -99,14 +95,13 @@ default MultiValueHandling getMultivalueHandling() * @return A new DimensionMergerV9 object. */ - DimensionMergerV9 makeMerger( + DimensionMergerV9 makeMerger( IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ); - /** * Given an key component representing a single set of row value(s) for this dimension as an Object, * return the length of the key component after appropriate type-casting. @@ -119,73 +114,17 @@ DimensionMergerV9 makeMerger( */ int getLengthOfEncodedKeyComponent(EncodedKeyComponentType dimVals); - - /** - * Given two key components representing sorted encoded row value(s), return the result of their comparison. - * - * If the two key components have different lengths, the shorter component should be ordered first in the comparison. - * - * Otherwise, this function should iterate through the key components and return the comparison of the - * first difference. - * - * For dimensions that do not support multivalue rows, lhs and rhs can be compared directly. - * - * @param lhs key component from a row - * @param rhs key component from a row - * - * @return integer indicating comparison result of key components - */ - int compareSortedEncodedKeyComponents(EncodedKeyComponentType lhs, EncodedKeyComponentType rhs); - - /** - * Given two key components representing sorted encoded row value(s), check that the two key components - * have the same encoded values, or if the encoded values differ, that they translate into the same actual values, - * using the mappings provided by lhsEncodings and rhsEncodings (if applicable). - * - * If validation fails, this method should throw a SegmentValidationException. - * - * Used by IndexIO for validating segments. - * - * See StringDimensionHandler.validateSortedEncodedKeyComponents() for a reference implementation. - * - * @param lhs key component from a row - * @param rhs key component from a row - * @param lhsEncodings encoding lookup from lhs's segment, null if not applicable for this dimension's type - * @param rhsEncodings encoding lookup from rhs's segment, null if not applicable for this dimension's type + * Returns a comparator that knows how to compare {@link ColumnValueSelector} of the assumed dimension type, + * corresponding to this DimensionHandler. E. g. {@link StringDimensionHandler} returns a comparator, that compares + * {@link ColumnValueSelector}s as {@link DimensionSelector}s. */ - void validateSortedEncodedKeyComponents( - EncodedKeyComponentType lhs, - EncodedKeyComponentType rhs, - Indexed lhsEncodings, - Indexed rhsEncodings - ) throws SegmentValidationException; - + Comparator getEncodedValueSelectorComparator(); /** - * Given a Column, return a type-specific object that can be used to retrieve row values. - * - * For example: - * - A String-typed implementation would return the result of column.getDictionaryEncoding() - * - A long-typed implemention would return the result of column.getGenericColumn(). - * - * @param column Column for this dimension from a QueryableIndex - * @return The type-specific column subobject for this dimension. - */ - Closeable getSubColumn(Column column); - - - /** - * Given a subcolumn from getSubColumn, and the index of the current row, retrieve a dimension's values - * from a row as an EncodedKeyComponentType. - * - * For example: - * - A String-typed implementation would read the current row from a DictionaryEncodedColumn as an int[]. - * - A long-typed implemention would read the current row from a GenericColumn and return a Long. - * - * @param column Column for this dimension from a QueryableIndex - * @param currRow The index of the row to retrieve - * @return The key component for this dimension from the current row of the column. + * Creates and returns a new object of some implementation of {@link SettableColumnValueSelector}, that corresponds + * to the type of this DimensionHandler. E. g. {@link LongDimensionHandler} returns {@link + * io.druid.segment.selector.settable.SettableLongColumnValueSelector}, etc. */ - EncodedKeyComponentType getEncodedKeyComponentFromColumn(Closeable column, int currRow); + SettableColumnValueSelector makeNewSettableEncodedValueSelector(); } diff --git a/processing/src/main/java/io/druid/segment/DimensionIndexer.java b/processing/src/main/java/io/druid/segment/DimensionIndexer.java index cbc298c2f246..444703abf659 100644 --- a/processing/src/main/java/io/druid/segment/DimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/DimensionIndexer.java @@ -24,7 +24,7 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; -import io.druid.segment.incremental.TimeAndDimsHolder; +import io.druid.segment.incremental.IncrementalIndexRowHolder; import javax.annotation.Nullable; @@ -32,7 +32,7 @@ * Processing related interface * * A DimensionIndexer is a per-dimension stateful object that encapsulates type-specific operations and data structures - * used during the in-memory ingestion process (i.e., work done by IncrementalIndex). + * used during the in-memory ingestion process (i.e., work done by {@link IncrementalIndex}). * * Ingested row values are passed to a DimensionIndexer, which will update its internal data structures such as * a value->ID dictionary as row values are seen. @@ -59,8 +59,8 @@ * The first ordering will be referred to as "Unsorted" in the documentation for this interface, and * the second ordering will be referred to as "Sorted". * - * The unsorted ordering is used during ingestion, within the IncrementalIndex's TimeAndDims keys; the encodings - * are built as rows are ingested, taking the order in which new dimension values are seen. + * The unsorted ordering is used during ingestion, within the {@link io.druid.segment.incremental.IncrementalIndexRow} + * keys; the encodings are built as rows are ingested, taking the order in which new dimension values are seen. * * The generation of a sorted encoding takes place during segment creation when indexes are merged/persisted. * The sorted ordering will be used for dimension value arrays in that context and when reading from @@ -112,7 +112,7 @@ public interface DimensionIndexer /** * Given a single row value or list of row values (for multi-valued dimensions), update any internal data structures - * with the ingested values and return the row values as an array to be used within a TimeAndDims key. + * with the ingested values and return the row values as an array to be used within a Row key. * * For example, the dictionary-encoded String-type column will return an int[] containing a dictionary ID. * @@ -193,29 +193,32 @@ public interface DimensionIndexer * Return an object used to read values from this indexer's column as Strings. * * @param spec Specifies the output name of a dimension and any extraction functions to be applied. - * @param currEntry Provides access to the current TimeAndDims object in the Cursor + * @param currEntry Provides access to the current Row object in the Cursor * @param desc Descriptor object for this dimension within an IncrementalIndex * @return A new object that reads rows from currEntry */ DimensionSelector makeDimensionSelector( DimensionSpec spec, - TimeAndDimsHolder currEntry, + IncrementalIndexRowHolder currEntry, IncrementalIndex.DimensionDesc desc ); /** * Return an object used to read values from this indexer's column. * - * @param currEntry Provides access to the current TimeAndDims object in the Cursor + * @param currEntry Provides access to the current Row object in the Cursor * @param desc Descriptor object for this dimension within an IncrementalIndex * @return A new object that reads rows from currEntry */ - ColumnValueSelector makeColumnValueSelector(TimeAndDimsHolder currEntry, IncrementalIndex.DimensionDesc desc); + ColumnValueSelector makeColumnValueSelector( + IncrementalIndexRowHolder currEntry, + IncrementalIndex.DimensionDesc desc + ); /** - * Compares the row values for this DimensionIndexer's dimension from a TimeAndDims key. + * Compares the row values for this DimensionIndexer's dimension from a Row key. * - * The dimension value arrays within a TimeAndDims key always use the "unsorted" ordering for encoded values. + * The dimension value arrays within a Row key always use the "unsorted" ordering for encoded values. * * The row values are passed to this function as an Object, the implementer should cast them to the type * appropriate for this dimension. @@ -230,26 +233,29 @@ DimensionSelector makeDimensionSelector( * * Refer to StringDimensionIndexer.compareUnsortedEncodedKeyComponents() for a reference implementation. * - * @param lhs dimension value array from a TimeAndDims key - * @param rhs dimension value array from a TimeAndDims key + * @param lhs dimension value array from a Row key + * @param rhs dimension value array from a Row key * @return comparison of the two arrays */ int compareUnsortedEncodedKeyComponents(@Nullable EncodedKeyComponentType lhs, @Nullable EncodedKeyComponentType rhs); /** - * Check if two row value arrays from TimeAndDims keys are equal. + * Check if two row value arrays from Row keys are equal. * - * @param lhs dimension value array from a TimeAndDims key - * @param rhs dimension value array from a TimeAndDims key + * @param lhs dimension value array from a Row key + * @param rhs dimension value array from a Row key * @return true if the two arrays are equal */ - boolean checkUnsortedEncodedKeyComponentsEqual(@Nullable EncodedKeyComponentType lhs, @Nullable EncodedKeyComponentType rhs); + boolean checkUnsortedEncodedKeyComponentsEqual( + @Nullable EncodedKeyComponentType lhs, + @Nullable EncodedKeyComponentType rhs + ); /** - * Given a row value array from a TimeAndDims key, generate a hashcode. - * @param key dimension value array from a TimeAndDims key + * Given a row value array from a Row key, generate a hashcode. + * @param key dimension value array from a Row key * @return hashcode of the array */ int getUnsortedEncodedKeyComponentHashCode(@Nullable EncodedKeyComponentType key); @@ -258,37 +264,33 @@ DimensionSelector makeDimensionSelector( boolean ARRAY = false; /** - * Given a row value array from a TimeAndDims key, as described in the documentation for + * Given a row value array from a Row key, as described in the documentation for * compareUnsortedEncodedKeyComponents(), convert the unsorted encoded values to a list or array of actual values. * * If the key has one element, this method should return a single Object instead of an array or list, ignoring * the asList parameter. * - * @param key dimension value array from a TimeAndDims key + * @param key dimension value array from a Row key * @param asList if true, return an array; if false, return a list * @return single value, array, or list containing the actual values corresponding to the encoded values * in the input array */ Object convertUnsortedEncodedKeyComponentToActualArrayOrList(EncodedKeyComponentType key, boolean asList); - /** - * Given a row value array from a TimeAndDims key, as described in the documentation for - * compareUnsortedEncodedKeyComponents(), convert the unsorted encoded values to an array of sorted encoded values - * (i.e., sorted by their corresponding actual values) - * - * @param key dimension value array from a TimeAndDims key - * @return array containing the sorted encoded values corresponding to the unsorted encoded values in the input array + * Converts dictionary-encoded row values from unspecified (random) encoding order, to sorted encoding. This step + * is needed to be able to correctly map per-segment encoded values to global values on the next conversion step, + * {@link DimensionMerger#convertSortedSegmentRowValuesToMergedRowValues}. The latter method requires sorted encoding + * values on the input, because {@link DimensionMerger#writeMergedValueDictionary} takes sorted lookups as it's input. */ - EncodedKeyComponentType convertUnsortedEncodedKeyComponentToSortedEncodedKeyComponent(EncodedKeyComponentType key); - + ColumnValueSelector convertUnsortedValuesToSorted(ColumnValueSelector selectorWithUnsortedValues); /** * Helper function for building bitmap indexes for integer-encoded dimensions. * * Called by IncrementalIndexAdapter as it iterates through its sequence of rows. * - * Given a row value array from a TimeAndDims key, with the current row number indicated by "rowNum", + * Given a row value array from a Row key, with the current row number indicated by "rowNum", * set the index for "rowNum" in the bitmap index for each value that appears in the row value array. * * For example, if key is an int[] array with values [1,3,4] for a dictionary-encoded String dimension, @@ -299,7 +301,7 @@ DimensionSelector makeDimensionSelector( * If a dimension type does not support bitmap indexes, this function will not be called * and can be left unimplemented. * - * @param key dimension value array from a TimeAndDims key + * @param key dimension value array from a Row key * @param rowNum current row number * @param bitmapIndexes array of bitmaps, indexed by integer dimension value * @param factory bitmap factory diff --git a/processing/src/main/java/io/druid/segment/DimensionMerger.java b/processing/src/main/java/io/druid/segment/DimensionMerger.java index 2c2b2dbbc992..d5140a594d0d 100644 --- a/processing/src/main/java/io/druid/segment/DimensionMerger.java +++ b/processing/src/main/java/io/druid/segment/DimensionMerger.java @@ -19,6 +19,7 @@ package io.druid.segment; +import javax.annotation.Nullable; import java.io.IOException; import java.nio.IntBuffer; import java.util.List; @@ -27,96 +28,85 @@ * Processing related interface * * A DimensionMerger is a per-dimension stateful object that encapsulates type-specific operations and data structures - * used during the segment merging process (i.e., work done by IndexMerger/IndexMergerV9). + * used during the segment merging process (i.e., work done by {@link IndexMerger}). * * This object is responsible for: - * - merging any relevant structures from the segments (e.g., encoding dictionaries) + * - merging encoding dictionaries, if present * - writing the merged column data and any merged indexing structures (e.g., dictionaries, bitmaps) to disk * * At a high level, the index merging process can be broken down into the following steps: - * - Merge any value representation metadata across segments - * E.g. for dictionary encoded Strings, each segment has its own unique space of id->value mappings. - * These need to be merged across segments into a shared space of dictionary mappings. + * - Merge segment's encoding dictionaries. These need to be merged across segments into a shared space of dictionary + * mappings: {@link #writeMergedValueDictionary(List)}. * - * - Merge the rows across segments into a common sequence of rows + * - Merge the rows across segments into a common sequence of rows. Done outside of scope of this interface, + * currently in {@link IndexMergerV9}. * - * - After constructing the merged sequence of rows, build any applicable index structures (e.g, bitmap indexes) + * - After constructing the merged sequence of rows, process each individual row via {@link #processMergedRow}, + * potentially continuing updating the internal structures. * - * - Write the value representation metadata (e.g. dictionary), the sequence of row values, - * and index structures to a merged segment. + * - Write the value representation metadata (dictionary, bitmaps), the sequence of row values, + * and index structures to a merged segment: {@link #writeIndexes} * * A class implementing this interface is expected to be highly stateful, updating its internal state as these * functions are called. - * - * @param A row key contains a component for each dimension, this param specifies the - * class of this dimension's key component. A column type that supports multivalue rows - * should use an array type (Strings would use int[]). Column types without multivalue - * row support should use single objects (e.g., Long, Float). */ -public interface DimensionMerger +public interface DimensionMerger { /** * Given a list of segment adapters: - * - Read any value metadata (e.g., dictionary encoding information) from the adapters - * - Merge this value metadata and update the internal state of the implementing class. + * - Read _sorted order_ (e. g. see {@link + * io.druid.segment.incremental.IncrementalIndexAdapter#getDimValueLookup(String)}) dictionary encoding information + * from the adapters + * - Merge those sorted order dictionary into a one big sorted order dictionary and write this merged dictionary. * * The implementer should maintain knowledge of the "index number" of the adapters in the input list, * i.e., the position of each adapter in the input list. * * This "index number" will be used to refer to specific segments later - * in convertSegmentRowValuesToMergedRowValues(). - * - * Otherwise, the details of how this merging occurs and how to store the merged data is left to the implementer. + * in {@link #convertSortedSegmentRowValuesToMergedRowValues}. * * @param adapters List of adapters to be merged. - * @throws IOException + * @see DimensionIndexer#convertUnsortedValuesToSorted */ - void writeMergedValueMetadata(List adapters) throws IOException; - + void writeMergedValueDictionary(List adapters) throws IOException; /** - * Convert a row's key component with per-segment encoding to its equivalent representation - * in the merged set of rows. + * Creates a value selector, which converts values with per-segment, _sorted order_ (see {@link + * DimensionIndexer#convertUnsortedValuesToSorted}) encoding from the given selector to their equivalent + * representation in the merged set of rows. * - * This function is used by the index merging process to build the merged sequence of rows. + * This method is used by the index merging process to build the merged sequence of rows. * * The implementing class is expected to use the merged value metadata constructed - * during writeMergedValueMetadata, if applicable. + * during {@link #writeMergedValueDictionary(List)}, if applicable. * * For example, an implementation of this function for a dictionary-encoded String column would convert the - * segment-specific dictionary values within the row to the common merged dictionary values - * determined during writeMergedValueMetadata(). + * segment-specific, sorted order dictionary values within the row to the common merged dictionary values + * determined during {@link #writeMergedValueDictionary(List)}. * - * @param segmentRow A row's key component for this dimension. The encoding of the key component's - * values will be converted from per-segment encodings to the combined encodings from - * the merged sequence of rows. - * @param segmentIndexNumber Integer indicating which segment the row originated from. + * @param segmentIndex indicates which segment the row originated from, in the order established in + * {@link #writeMergedValueDictionary(List)} + * @param source the selector from which to take values to convert + * @return a selector with converted values */ - EncodedKeyComponentType convertSegmentRowValuesToMergedRowValues( - EncodedKeyComponentType segmentRow, - int segmentIndexNumber - ); - + ColumnValueSelector convertSortedSegmentRowValuesToMergedRowValues(int segmentIndex, ColumnValueSelector source); /** - * Process a key component from the merged sequence of rows and update the DimensionMerger's internal state. + * Process a column value(s) (potentially multi-value) of a row from the given selector and update the + * DimensionMerger's internal state. * - * After constructing a merged sequence of rows across segments, the index merging process will - * iterate through these rows and pass row key components from each dimension to their corresponding DimensionMergers. + * After constructing a merged sequence of rows across segments, the index merging process will iterate through these + * rows and on each iteration, for each column, pass the column value selector to the corresponding DimensionMerger. * * This allows each DimensionMerger to build its internal view of the sequence of merged rows, to be * written out to a segment later. - * - * @param rowValues The row values to be added. - * @throws IOException */ - void processMergedRow(EncodedKeyComponentType rowValues) throws IOException; - + void processMergedRow(ColumnValueSelector selector) throws IOException; /** * Internally construct any index structures relevant to this DimensionMerger. * - * After receiving the sequence of merged rows via iterated processMergedRow() calls, the DimensionMerger + * After receiving the sequence of merged rows via iterated {@link #processMergedRow} calls, the DimensionMerger * can now build any index structures it needs. * * For example, a dictionary encoded String implementation would create its bitmap indexes @@ -124,17 +114,15 @@ EncodedKeyComponentType convertSegmentRowValuesToMergedRowValues( * * The index merger will provide a list of row number conversion IntBuffer objects. * Each IntBuffer is associated with one of the segments being merged; the position of the IntBuffer in the list - * corresponds to the position of segment adapters within the input list of writeMergedValueMetadata(). + * corresponds to the position of segment adapters within the input list of {@link #writeMergedValueDictionary(List)}. * * For example, suppose there are two segments A and B. * Row 24 from segment A maps to row 99 in the merged sequence of rows, * The IntBuffer for segment A would have a mapping of 24 -> 99. * * @param segmentRowNumConversions A list of row number conversion IntBuffer objects. - * @throws IOException */ - void writeIndexes(List segmentRowNumConversions) throws IOException; - + void writeIndexes(@Nullable List segmentRowNumConversions) throws IOException; /** * Return true if this dimension's data does not need to be written to the segment. diff --git a/processing/src/main/java/io/druid/segment/DimensionMergerV9.java b/processing/src/main/java/io/druid/segment/DimensionMergerV9.java index bb00e847d70c..7cd1e22c9d46 100644 --- a/processing/src/main/java/io/druid/segment/DimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/DimensionMergerV9.java @@ -26,7 +26,7 @@ * * DimensionMerger subclass to be used with IndexMergerV9. */ -public interface DimensionMergerV9 extends DimensionMerger +public interface DimensionMergerV9 extends DimensionMerger { /** * Return a ColumnDescriptor containing ColumnPartSerde objects appropriate for diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java b/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java index e6b6df6098f8..fda831d27a07 100644 --- a/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java @@ -73,10 +73,9 @@ public void open() throws IOException } @Override - public void serialize(Object obj) throws IOException + public void serialize(ColumnValueSelector selector) throws IOException { - double val = (obj == null) ? 0 : ((Number) obj).doubleValue(); - writer.add(val); + writer.add(selector.getDouble()); } @Override diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java index 07828f996cba..81162826695d 100644 --- a/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java @@ -32,7 +32,6 @@ import io.druid.segment.data.CompressionStrategy; import io.druid.segment.writeout.SegmentWriteOutMedium; -import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; @@ -106,13 +105,13 @@ public void open() throws IOException } @Override - public void serialize(@Nullable Object obj) throws IOException + public void serialize(ColumnValueSelector selector) throws IOException { - if (obj == null) { + if (selector.isNull()) { nullRowsBitmap.add(rowCount); writer.add(0D); } else { - writer.add(((Number) obj).doubleValue()); + writer.add(selector.getDouble()); } rowCount++; } diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java b/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java index aac925fce7d7..57c7237637e6 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java @@ -19,16 +19,25 @@ package io.druid.segment; -import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.GenericColumn; -import io.druid.segment.data.Indexed; +import io.druid.segment.selector.settable.SettableColumnValueSelector; +import io.druid.segment.selector.settable.SettableDoubleColumnValueSelector; import io.druid.segment.writeout.SegmentWriteOutMedium; -import java.io.Closeable; +import java.util.Comparator; public class DoubleDimensionHandler implements DimensionHandler { + private static Comparator DOUBLE_COLUMN_COMPARATOR = (s1, s2) -> { + if (s1.isNull()) { + return s2.isNull() ? 0 : -1; + } else if (s2.isNull()) { + return 1; + } else { + return Double.compare(s1.getDouble(), s2.getDouble()); + } + }; + private final String dimensionName; public DoubleDimensionHandler(String dimensionName) @@ -49,7 +58,7 @@ public DimensionIndexer makeIndexer() } @Override - public DimensionMergerV9 makeMerger( + public DimensionMergerV9 makeMerger( IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -70,35 +79,14 @@ public int getLengthOfEncodedKeyComponent(Double dimVals) } @Override - public int compareSortedEncodedKeyComponents(Double lhs, Double rhs) - { - return lhs.compareTo(rhs); - } - - @Override - public void validateSortedEncodedKeyComponents( - Double lhs, Double rhs, Indexed lhsEncodings, Indexed rhsEncodings - ) throws SegmentValidationException - { - if (!lhs.equals(rhs)) { - throw new SegmentValidationException( - "Dim [%s] value not equal. Expected [%s] found [%s]", - dimensionName, - lhs, - rhs - ); - } - } - - @Override - public Closeable getSubColumn(Column column) + public Comparator getEncodedValueSelectorComparator() { - return column.getGenericColumn(); + return DOUBLE_COLUMN_COMPARATOR; } @Override - public Double getEncodedKeyComponentFromColumn(Closeable column, int currRow) + public SettableColumnValueSelector makeNewSettableEncodedValueSelector() { - return ((GenericColumn) column).getDoubleSingleValueRow(currRow); + return new SettableDoubleColumnValueSelector(); } } diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java index a0a7f30e31f3..7449e75df7a1 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java @@ -27,7 +27,7 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; -import io.druid.segment.incremental.TimeAndDimsHolder; +import io.druid.segment.incremental.IncrementalIndexRowHolder; import javax.annotation.Nullable; import java.util.Comparator; @@ -82,7 +82,7 @@ public int getCardinality() @Override public DimensionSelector makeDimensionSelector( DimensionSpec spec, - TimeAndDimsHolder currEntry, + IncrementalIndexRowHolder currEntry, IncrementalIndex.DimensionDesc desc ) { @@ -91,7 +91,7 @@ public DimensionSelector makeDimensionSelector( @Override public ColumnValueSelector makeColumnValueSelector( - TimeAndDimsHolder currEntry, + IncrementalIndexRowHolder currEntry, IncrementalIndex.DimensionDesc desc ) { @@ -166,9 +166,9 @@ public Object convertUnsortedEncodedKeyComponentToActualArrayOrList(Double key, } @Override - public Double convertUnsortedEncodedKeyComponentToSortedEncodedKeyComponent(Double key) + public ColumnValueSelector convertUnsortedValuesToSorted(ColumnValueSelector selectorWithUnsortedValues) { - return key; + return selectorWithUnsortedValues; } @Override diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java index 032febbf334e..0fd10d42bbac 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java @@ -24,72 +24,18 @@ import io.druid.segment.serde.ColumnPartSerde; import io.druid.segment.writeout.SegmentWriteOutMedium; -import java.io.IOException; -import java.nio.IntBuffer; -import java.util.List; - -public class DoubleDimensionMergerV9 implements DimensionMergerV9 +public class DoubleDimensionMergerV9 extends NumericDimensionMergerV9 { - protected String dimensionName; - protected final IndexSpec indexSpec; - private GenericColumnSerializer serializer; - - public DoubleDimensionMergerV9( - String dimensionName, - IndexSpec indexSpec, - SegmentWriteOutMedium segmentWriteOutMedium - ) - { - this.dimensionName = dimensionName; - this.indexSpec = indexSpec; - - try { - setupEncodedValueWriter(segmentWriteOutMedium); - } - catch (IOException ioe) { - throw new RuntimeException(ioe); - } - } - - private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException - { - this.serializer = IndexMergerV9.createDoubleColumnSerializer( - segmentWriteOutMedium, - dimensionName, - indexSpec - ); - - serializer.open(); - } - - @Override - public void writeMergedValueMetadata(List adapters) - { - // double columns do not have additional metadata - } - - @Override - public Double convertSegmentRowValuesToMergedRowValues(Double segmentRow, int segmentIndexNumber) - { - return segmentRow; - } - @Override - public void processMergedRow(Double rowValues) throws IOException - { - serializer.serialize(rowValues); - } - - @Override - public void writeIndexes(List segmentRowNumConversions) + DoubleDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) { - // double columns do not have indexes + super(dimensionName, indexSpec, segmentWriteOutMedium); } @Override - public boolean canSkip() + GenericColumnSerializer setupEncodedValueWriter() { - return false; + return IndexMergerV9.createDoubleColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec); } @Override diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java index 695d20871cad..7e896f516f06 100644 --- a/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java @@ -73,10 +73,9 @@ public void open() throws IOException } @Override - public void serialize(Object obj) throws IOException + public void serialize(ColumnValueSelector selector) throws IOException { - float val = (obj == null) ? 0 : ((Number) obj).floatValue(); - writer.add(val); + writer.add(selector.getFloat()); } @Override diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java index 1ec6f377e713..366d26e20ea4 100644 --- a/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java +++ b/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java @@ -32,7 +32,6 @@ import io.druid.segment.data.CompressionStrategy; import io.druid.segment.writeout.SegmentWriteOutMedium; -import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; @@ -106,19 +105,18 @@ public void open() throws IOException } @Override - public void serialize(@Nullable Object obj) throws IOException + public void serialize(ColumnValueSelector selector) throws IOException { - if (obj == null) { + if (selector.isNull()) { nullRowsBitmap.add(rowCount); writer.add(0f); } else { - writer.add(((Number) obj).floatValue()); + writer.add(selector.getFloat()); } rowCount++; } @Override - public long getSerializedSize() throws IOException { nullValueBitmapWriter.write(bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(nullRowsBitmap)); diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java b/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java index 7e63f55044c8..5dae68483d92 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java @@ -19,16 +19,25 @@ package io.druid.segment; -import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.GenericColumn; -import io.druid.segment.data.Indexed; +import io.druid.segment.selector.settable.SettableColumnValueSelector; +import io.druid.segment.selector.settable.SettableFloatColumnValueSelector; import io.druid.segment.writeout.SegmentWriteOutMedium; -import java.io.Closeable; +import java.util.Comparator; public class FloatDimensionHandler implements DimensionHandler { + private static Comparator FLOAT_COLUMN_COMPARATOR = (s1, s2) -> { + if (s1.isNull()) { + return s2.isNull() ? 0 : -1; + } else if (s2.isNull()) { + return 1; + } else { + return Float.compare(s1.getFloat(), s2.getFloat()); + } + }; + private final String dimensionName; public FloatDimensionHandler(String dimensionName) @@ -49,7 +58,7 @@ public DimensionIndexer makeIndexer() } @Override - public DimensionMergerV9 makeMerger( + public DimensionMergerV9 makeMerger( IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -70,35 +79,14 @@ public int getLengthOfEncodedKeyComponent(Float dimVals) } @Override - public int compareSortedEncodedKeyComponents(Float lhs, Float rhs) - { - return lhs.compareTo(rhs); - } - - @Override - public void validateSortedEncodedKeyComponents( - Float lhs, Float rhs, Indexed lhsEncodings, Indexed rhsEncodings - ) throws SegmentValidationException - { - if (!lhs.equals(rhs)) { - throw new SegmentValidationException( - "Dim [%s] value not equal. Expected [%s] found [%s]", - dimensionName, - lhs, - rhs - ); - } - } - - @Override - public Closeable getSubColumn(Column column) + public Comparator getEncodedValueSelectorComparator() { - return column.getGenericColumn(); + return FLOAT_COLUMN_COMPARATOR; } @Override - public Float getEncodedKeyComponentFromColumn(Closeable column, int currRow) + public SettableColumnValueSelector makeNewSettableEncodedValueSelector() { - return ((GenericColumn) column).getFloatSingleValueRow(currRow); + return new SettableFloatColumnValueSelector(); } } diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java index 695535cdac5a..cdcc43188972 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java @@ -27,7 +27,7 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; -import io.druid.segment.incremental.TimeAndDimsHolder; +import io.druid.segment.incremental.IncrementalIndexRowHolder; import javax.annotation.Nullable; import java.util.Comparator; @@ -83,7 +83,7 @@ public int getCardinality() @Override public DimensionSelector makeDimensionSelector( DimensionSpec spec, - TimeAndDimsHolder currEntry, + IncrementalIndexRowHolder currEntry, IncrementalIndex.DimensionDesc desc ) { @@ -92,7 +92,7 @@ public DimensionSelector makeDimensionSelector( @Override public ColumnValueSelector makeColumnValueSelector( - TimeAndDimsHolder currEntry, + IncrementalIndexRowHolder currEntry, IncrementalIndex.DimensionDesc desc ) { @@ -169,9 +169,9 @@ public Object convertUnsortedEncodedKeyComponentToActualArrayOrList(Float key, b } @Override - public Float convertUnsortedEncodedKeyComponentToSortedEncodedKeyComponent(Float key) + public ColumnValueSelector convertUnsortedValuesToSorted(ColumnValueSelector selectorWithUnsortedValues) { - return key; + return selectorWithUnsortedValues; } @Override diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java index 31b691d8ec91..f49e7415e4b3 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java @@ -24,72 +24,18 @@ import io.druid.segment.serde.ColumnPartSerde; import io.druid.segment.writeout.SegmentWriteOutMedium; -import java.io.IOException; -import java.nio.IntBuffer; -import java.util.List; - -public class FloatDimensionMergerV9 implements DimensionMergerV9 +public class FloatDimensionMergerV9 extends NumericDimensionMergerV9 { - protected String dimensionName; - protected final IndexSpec indexSpec; - private GenericColumnSerializer serializer; - - public FloatDimensionMergerV9( - String dimensionName, - IndexSpec indexSpec, - SegmentWriteOutMedium segmentWriteOutMedium - ) - { - this.dimensionName = dimensionName; - this.indexSpec = indexSpec; - try { - setupEncodedValueWriter(segmentWriteOutMedium); - } - catch (IOException ioe) { - throw new RuntimeException(ioe); - } - } - - private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException - { - this.serializer = IndexMergerV9.createFloatColumnSerializer( - segmentWriteOutMedium, - dimensionName, - indexSpec - ); - serializer.open(); - } - - @Override - public void writeMergedValueMetadata(List adapters) - { - // floats have no additional metadata - } - - @Override - public Float convertSegmentRowValuesToMergedRowValues(Float segmentRow, int segmentIndexNumber) - { - return segmentRow; - } - - @Override - public void processMergedRow(Float rowValues) throws IOException - { - serializer.serialize(rowValues); - } - - @Override - public void writeIndexes(List segmentRowNumConversions) + FloatDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) { - // floats have no indices to write + super(dimensionName, indexSpec, segmentWriteOutMedium); } @Override - public boolean canSkip() + GenericColumnSerializer setupEncodedValueWriter() { - // a float column can never be all null - return false; + return IndexMergerV9.createFloatColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec); } @Override diff --git a/processing/src/main/java/io/druid/segment/ForwardingRowIterator.java b/processing/src/main/java/io/druid/segment/ForwardingRowIterator.java new file mode 100644 index 000000000000..965b83040c72 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/ForwardingRowIterator.java @@ -0,0 +1,70 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +/** + * Implementation of {@link TransformableRowIterator} that just delegates all methods to some baseIterator (adapter + * pattern). Subclass should override some methods. + */ +public abstract class ForwardingRowIterator implements TransformableRowIterator +{ + protected final TransformableRowIterator baseIterator; + + protected ForwardingRowIterator(TransformableRowIterator baseIterator) + { + this.baseIterator = baseIterator; + } + + @Override + public void mark() + { + baseIterator.mark(); + } + + @Override + public TimeAndDimsPointer getMarkedPointer() + { + return baseIterator.getMarkedPointer(); + } + + @Override + public boolean hasTimeAndDimsChangedSinceMark() + { + return baseIterator.hasTimeAndDimsChangedSinceMark(); + } + + @Override + public RowPointer getPointer() + { + return baseIterator.getPointer(); + } + + @Override + public boolean moveToNext() + { + return baseIterator.moveToNext(); + } + + @Override + public void close() + { + baseIterator.close(); + } +} diff --git a/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java b/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java index a9f35004fab5..1e3ba907f5e7 100644 --- a/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java @@ -29,5 +29,5 @@ public interface GenericColumnSerializer extends Serializer { void open() throws IOException; - void serialize(Object obj) throws IOException; + void serialize(ColumnValueSelector selector) throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 88d90055943a..890a0e09249a 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -32,7 +32,6 @@ import com.google.common.io.Files; import com.google.common.primitives.Ints; import com.google.inject.Inject; -import io.druid.java.util.emitter.EmittingLogger; import io.druid.collections.bitmap.ConciseBitmapFactory; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.spatial.ImmutableRTree; @@ -45,6 +44,7 @@ import io.druid.java.util.common.io.smoosh.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.java.util.common.logger.Logger; +import io.druid.java.util.emitter.EmittingLogger; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnCapabilities; @@ -56,7 +56,6 @@ import io.druid.segment.data.CompressedColumnarLongsSupplier; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.ImmutableRTreeObjectStrategy; -import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.VSizeColumnarMultiInts; import io.druid.segment.serde.BitmapIndexColumnPartSupplier; @@ -76,8 +75,9 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Arrays; -import java.util.Iterator; +import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; public class IndexIO @@ -98,10 +98,15 @@ public class IndexIO private final SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory; @Inject - public IndexIO(ObjectMapper mapper, SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory, ColumnConfig columnConfig) + public IndexIO( + ObjectMapper mapper, + SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory, + ColumnConfig columnConfig + ) { this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); - this.defaultSegmentWriteOutMediumFactory = Preconditions.checkNotNull(defaultSegmentWriteOutMediumFactory, "null SegmentWriteOutMediumFactory"); + this.defaultSegmentWriteOutMediumFactory = + Preconditions.checkNotNull(defaultSegmentWriteOutMediumFactory, "null SegmentWriteOutMediumFactory"); Preconditions.checkNotNull(columnConfig, "null ColumnConfig"); ImmutableMap.Builder indexLoadersBuilder = ImmutableMap.builder(); LegacyIndexLoader legacyIndexLoader = new LegacyIndexLoader(new DefaultIndexIOHandler(), columnConfig); @@ -149,29 +154,27 @@ public void validateTwoSegments(final IndexableAdapter adapter1, final Indexable throw new SegmentValidationException("Metric names differ. Expected [%s] found [%s]", metNames1, metNames2); } } - final Map dimHandlers = adapter1.getDimensionHandlers(); - - final Iterator it1 = adapter1.getRows().iterator(); - final Iterator it2 = adapter2.getRows().iterator(); + final RowIterator it1 = adapter1.getRows(); + final RowIterator it2 = adapter2.getRows(); long row = 0L; - while (it1.hasNext()) { - if (!it2.hasNext()) { + while (it1.moveToNext()) { + if (!it2.moveToNext()) { throw new SegmentValidationException("Unexpected end of second adapter"); } - final Rowboat rb1 = it1.next(); - final Rowboat rb2 = it2.next(); + final RowPointer rp1 = it1.getPointer(); + final RowPointer rp2 = it2.getPointer(); ++row; - if (rb1.getRowNum() != rb2.getRowNum()) { - throw new SegmentValidationException("Row number mismatch: [%d] vs [%d]", rb1.getRowNum(), rb2.getRowNum()); + if (rp1.getRowNum() != rp2.getRowNum()) { + throw new SegmentValidationException("Row number mismatch: [%d] vs [%d]", rp1.getRowNum(), rp2.getRowNum()); } try { - validateRowValues(dimHandlers, rb1, adapter1, rb2, adapter2); + validateRowValues(rp1, adapter1, rp2, adapter2); } catch (SegmentValidationException ex) { - throw new SegmentValidationException(ex, "Validation failure on row %d: [%s] vs [%s]", row, rb1, rb2); + throw new SegmentValidationException(ex, "Validation failure on row %d: [%s] vs [%s]", row, rp1, rp2); } } - if (it2.hasNext()) { + if (it2.moveToNext()) { throw new SegmentValidationException("Unexpected end of first adapter"); } if (row != adapter1.getNumRows()) { @@ -250,22 +253,22 @@ interface IndexIOHandler MMappedIndex mapDir(File inDir) throws IOException; } - public static void validateRowValues( - Map dimHandlers, - Rowboat rb1, + private static void validateRowValues( + RowPointer rp1, IndexableAdapter adapter1, - Rowboat rb2, + RowPointer rp2, IndexableAdapter adapter2 ) { - if (rb1.getTimestamp() != rb2.getTimestamp()) { + if (rp1.getTimestamp() != rp2.getTimestamp()) { throw new SegmentValidationException( "Timestamp mismatch. Expected %d found %d", - rb1.getTimestamp(), rb2.getTimestamp() + rp1.getTimestamp(), + rp2.getTimestamp() ); } - final Object[] dims1 = rb1.getDims(); - final Object[] dims2 = rb2.getDims(); + final Object[] dims1 = rp1.getDimensionValuesForDebug(); + final Object[] dims2 = rp2.getDimensionValuesForDebug(); if (dims1.length != dims2.length) { throw new SegmentValidationException( "Dim lengths not equal %s vs %s", @@ -273,11 +276,9 @@ public static void validateRowValues( Arrays.deepToString(dims2) ); } - final Indexed dim1Names = adapter1.getDimensionNames(); - final Indexed dim2Names = adapter2.getDimensionNames(); + final List dim1Names = adapter1.getDimensionNames(); + final List dim2Names = adapter2.getDimensionNames(); for (int i = 0; i < dims1.length; ++i) { - final Object dim1Vals = dims1[i]; - final Object dim2Vals = dims2[i]; final String dim1Name = dim1Names.get(i); final String dim2Name = dim2Names.get(i); @@ -294,14 +295,51 @@ public static void validateRowValues( ); } - DimensionHandler dimHandler = dimHandlers.get(dim1Name); - dimHandler.validateSortedEncodedKeyComponents( - dim1Vals, - dim2Vals, - adapter1.getDimValueLookup(dim1Name), - adapter2.getDimValueLookup(dim2Name) - ); + Object vals1 = dims1[i]; + Object vals2 = dims2[i]; + if (isNullRow(vals1) ^ isNullRow(vals2)) { + throw notEqualValidationException(dim1Name, vals1, vals2); + } + if (vals1 instanceof Object[] && vals2 instanceof Object[]) { + if (!Arrays.equals((Object[]) vals1, (Object[]) vals2)) { + throw notEqualValidationException(dim1Name, vals1, vals2); + } + } else if (vals1 instanceof Object[]) { + if (((Object[]) vals1).length != 1 || !Objects.equals(((Object[]) vals1)[0], vals2)) { + throw notEqualValidationException(dim1Name, vals1, vals2); + } + } else if (vals2 instanceof Object[]) { + if (((Object[]) vals2).length != 1 || !Objects.equals(((Object[]) vals2)[0], vals1)) { + throw notEqualValidationException(dim1Name, vals1, vals2); + } + } else { + if (!Objects.equals(vals1, vals2)) { + throw notEqualValidationException(dim1Name, vals1, vals2); + } + } + } + } + + private static boolean isNullRow(@Nullable Object row) + { + if (row == null) { + return true; + } + if (!(row instanceof Object[])) { + return false; } + for (Object v : (Object[]) row) { + //noinspection VariableNotUsedInsideIf + if (v != null) { + return false; + } + } + return true; + } + + private static SegmentValidationException notEqualValidationException(String dimName, Object v1, Object v2) + { + return new SegmentValidationException("Dim [%s] values not equal. Expected %s found %s", dimName, v1, v2); } public static class DefaultIndexIOHandler implements IndexIOHandler @@ -458,11 +496,7 @@ public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException ) ); if (index.getSpatialIndexes().get(dimension) != null) { - builder.setSpatialIndex( - new SpatialIndexColumnPartSupplier( - index.getSpatialIndexes().get(dimension) - ) - ); + builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(index.getSpatialIndexes().get(dimension))); } columns.put( dimension, @@ -477,11 +511,12 @@ public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException metric, new ColumnBuilder() .setType(ValueType.FLOAT) - .setGenericColumn(new FloatGenericColumnSupplier( - metricHolder.floatType, - LEGACY_FACTORY.getBitmapFactory() - .makeEmptyImmutableBitmap() - )) + .setGenericColumn( + new FloatGenericColumnSupplier( + metricHolder.floatType, + LEGACY_FACTORY.getBitmapFactory().makeEmptyImmutableBitmap() + ) + ) .build() ); } else if (metricHolder.getType() == MetricHolder.MetricType.COMPLEX) { @@ -491,7 +526,8 @@ public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException .setType(ValueType.COMPLEX) .setComplexColumn( new ComplexColumnPartSupplier( - metricHolder.getTypeName(), (GenericIndexed) metricHolder.complexType + metricHolder.getTypeName(), + (GenericIndexed) metricHolder.complexType ) ) .build() @@ -511,11 +547,12 @@ public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException Column.TIME_COLUMN_NAME, new ColumnBuilder() .setType(ValueType.LONG) - .setGenericColumn(new LongGenericColumnSupplier( - index.timestamps, - LEGACY_FACTORY.getBitmapFactory() - .makeEmptyImmutableBitmap() - )) + .setGenericColumn( + new LongGenericColumnSupplier( + index.timestamps, + LEGACY_FACTORY.getBitmapFactory().makeEmptyImmutableBitmap() + ) + ) .build() ); return new SimpleQueryableIndex( diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 75150b767f50..8dad991388f1 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; import com.google.common.collect.PeekingIterator; import com.google.common.collect.Sets; import com.google.inject.ImplementedBy; @@ -34,16 +33,12 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.guava.Comparators; -import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.writeout.SegmentWriteOutMediumFactory; -import it.unimi.dsi.fastutil.ints.Int2ObjectMap; -import it.unimi.dsi.fastutil.ints.IntIterator; -import it.unimi.dsi.fastutil.ints.IntSortedSet; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -52,7 +47,7 @@ import java.nio.ByteBuffer; import java.nio.IntBuffer; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -305,125 +300,93 @@ public int seek(int dictId) } } - class MMappedIndexRowIterable implements Iterable + /** + * This method applies {@link DimensionMerger#convertSortedSegmentRowValuesToMergedRowValues(int, ColumnValueSelector)} to + * all dimension column selectors of the given sourceRowIterator, using the given index number. + */ + static TransformableRowIterator toMergedIndexRowIterator( + TransformableRowIterator sourceRowIterator, + int indexNumber, + final List mergers + ) { - private final Iterable index; - private final List convertedDims; - private final int indexNumber; - private final List mergers; - - - MMappedIndexRowIterable( - Iterable index, - List convertedDims, - int indexNumber, - final List mergers - ) - { - this.index = index; - this.convertedDims = convertedDims; - this.indexNumber = indexNumber; - this.mergers = mergers; - } - - @Override - public Iterator iterator() - { - return Iterators.transform( - index.iterator(), - new Function() - { - @Override - public Rowboat apply(@Nullable Rowboat input) - { - Object[] dims = input.getDims(); - Object[] newDims = new Object[convertedDims.size()]; - for (int i = 0; i < convertedDims.size(); ++i) { - if (i >= dims.length) { - continue; - } - newDims[i] = mergers.get(i).convertSegmentRowValuesToMergedRowValues(dims[i], indexNumber); - } - - final Rowboat retVal = new Rowboat( - input.getTimestamp(), - newDims, - input.getMetrics(), - input.getRowNum(), - input.getHandlers() - ); - - retVal.addRow(indexNumber, input.getRowNum()); - - return retVal; - } - } + RowPointer sourceRowPointer = sourceRowIterator.getPointer(); + TimeAndDimsPointer markedSourceRowPointer = sourceRowIterator.getMarkedPointer(); + boolean anySelectorChanged = false; + ColumnValueSelector[] convertedDimensionSelectors = new ColumnValueSelector[mergers.size()]; + ColumnValueSelector[] convertedMarkedDimensionSelectors = new ColumnValueSelector[mergers.size()]; + for (int i = 0; i < mergers.size(); i++) { + ColumnValueSelector sourceDimensionSelector = sourceRowPointer.getDimensionSelector(i); + ColumnValueSelector convertedDimensionSelector = + mergers.get(i).convertSortedSegmentRowValuesToMergedRowValues(indexNumber, sourceDimensionSelector); + convertedDimensionSelectors[i] = convertedDimensionSelector; + // convertedDimensionSelector could be just the same object as sourceDimensionSelector, it means that this + // type of column doesn't have any kind of special per-index encoding that needs to be converted to the "global" + // encoding. E. g. it's always true for subclasses of NumericDimensionMergerV9. + //noinspection ObjectEquality + anySelectorChanged |= convertedDimensionSelector != sourceDimensionSelector; + + convertedMarkedDimensionSelectors[i] = mergers.get(i).convertSortedSegmentRowValuesToMergedRowValues( + indexNumber, + markedSourceRowPointer.getDimensionSelector(i) ); } + // If none dimensions are actually converted, don't need to transform the sourceRowIterator, adding extra + // indirection layer. It could be just returned back from this method. + if (!anySelectorChanged) { + return sourceRowIterator; + } + return makeRowIteratorWithConvertedDimensionColumns( + sourceRowIterator, + convertedDimensionSelectors, + convertedMarkedDimensionSelectors + ); } - class RowboatMergeFunction implements BinaryFn + static TransformableRowIterator makeRowIteratorWithConvertedDimensionColumns( + TransformableRowIterator sourceRowIterator, + ColumnValueSelector[] convertedDimensionSelectors, + ColumnValueSelector[] convertedMarkedDimensionSelectors + ) { - private final AggregatorFactory[] metricAggs; - - public RowboatMergeFunction(AggregatorFactory[] metricAggs) + RowPointer convertedRowPointer = sourceRowIterator.getPointer().withDimensionSelectors(convertedDimensionSelectors); + TimeAndDimsPointer convertedMarkedRowPointer = + sourceRowIterator.getMarkedPointer().withDimensionSelectors(convertedMarkedDimensionSelectors); + return new ForwardingRowIterator(sourceRowIterator) { - this.metricAggs = metricAggs; - } - - @Override - public Rowboat apply(Rowboat lhs, Rowboat rhs) - { - if (lhs == null) { - return rhs; - } - if (rhs == null) { - return lhs; + @Override + public RowPointer getPointer() + { + return convertedRowPointer; } - Object[] metrics = new Object[metricAggs.length]; - Object[] lhsMetrics = lhs.getMetrics(); - Object[] rhsMetrics = rhs.getMetrics(); - - for (int i = 0; i < metrics.length; ++i) { - Object lhsMetric = lhsMetrics[i]; - Object rhsMetric = rhsMetrics[i]; - if (lhsMetric == null) { - metrics[i] = rhsMetric; - } else if (rhsMetric == null) { - metrics[i] = lhsMetric; - } else { - metrics[i] = metricAggs[i].combine(lhsMetric, rhsMetric); - } - } - - final Rowboat retVal = new Rowboat( - lhs.getTimestamp(), - lhs.getDims(), - metrics, - lhs.getRowNum(), - lhs.getHandlers() - ); - - for (Rowboat rowboat : Arrays.asList(lhs, rhs)) { - Iterator> entryIterator = - rowboat.getComprisedRows().int2ObjectEntrySet().fastIterator(); - while (entryIterator.hasNext()) { - Int2ObjectMap.Entry entry = entryIterator.next(); - - for (IntIterator setIterator = entry.getValue().iterator(); setIterator.hasNext(); /* NOP */) { - int rowNum = setIterator.nextInt(); - retVal.addRow(entry.getIntKey(), rowNum); - } - } + @Override + public TimeAndDimsPointer getMarkedPointer() + { + return convertedMarkedRowPointer; } - - return retVal; - } + }; } class DictionaryMergeIterator implements CloseableIterator { + /** + * Don't replace this lambda with {@link Comparator#comparing} or {@link Comparators#naturalNullsFirst()} because + * this comparator is hot, so we want to avoid extra indirection layers. + */ + static final Comparator>> NULLS_FIRST_PEEKING_COMPARATOR = (lhs, rhs) -> { + String left = lhs.rhs.peek(); + String right = rhs.rhs.peek(); + if (left == null) { + //noinspection VariableNotUsedInsideIf + return right == null ? 0 : -1; + } else if (right == null) { + return 1; + } else { + return left.compareTo(right); + } + }; + protected final IntBuffer[] conversions; protected final List> directBufferAllocations = Lists.newArrayList(); protected final PriorityQueue>> pQueue; @@ -432,11 +395,7 @@ class DictionaryMergeIterator implements CloseableIterator DictionaryMergeIterator(Indexed[] dimValueLookups, boolean useDirect) { - final Ordering stringOrdering = Comparators.naturalNullsFirst(); - pQueue = new PriorityQueue<>( - dimValueLookups.length, - (lhs, rhs) -> stringOrdering.compare(lhs.rhs.peek(), rhs.rhs.peek()) - ); + pQueue = new PriorityQueue<>(dimValueLookups.length, NULLS_FIRST_PEEKING_COMPARATOR); conversions = new IntBuffer[dimValueLookups.length]; for (int i = 0; i < conversions.length; i++) { if (dimValueLookups[i] == null) { diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index 00e7e39021bf..87af6a20d4e8 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -20,19 +20,15 @@ package io.druid.segment; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.common.io.Files; import com.google.common.primitives.Ints; -import com.google.common.primitives.Longs; import com.google.inject.Inject; -import io.druid.collections.CombiningIterable; import io.druid.common.config.NullHandling; import io.druid.io.ZeroCopyByteArrayOutputStream; import io.druid.java.util.common.DateTimes; @@ -40,8 +36,6 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.guava.Comparators; -import io.druid.java.util.common.guava.FunctionalIterable; -import io.druid.java.util.common.guava.MergeIterable; import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.SmooshedWriter; @@ -53,7 +47,6 @@ import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; import io.druid.segment.loading.MMappedQueryableSegmentizerFactory; @@ -69,9 +62,6 @@ import io.druid.segment.serde.LongGenericColumnPartSerdeV2; import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.writeout.SegmentWriteOutMediumFactory; -import it.unimi.dsi.fastutil.ints.Int2ObjectMap; -import it.unimi.dsi.fastutil.ints.IntIterator; -import it.unimi.dsi.fastutil.ints.IntSortedSet; import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -84,10 +74,12 @@ import java.nio.IntBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; public class IndexMergerV9 implements IndexMerger { @@ -113,7 +105,8 @@ private File makeIndexFiles( final ProgressIndicator progress, final List mergedDimensions, final List mergedMetrics, - final Function>, Iterable> rowMergerFn, + final Function, TimeAndDimsIterator> rowMergerFn, + final boolean fillRowNumConversions, final IndexSpec indexSpec, final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException @@ -121,17 +114,7 @@ private File makeIndexFiles( progress.start(); progress.progress(); - List metadataList = Lists.transform( - adapters, - new Function() - { - @Override - public Metadata apply(IndexableAdapter input) - { - return input.getMetadata(); - } - } - ); + List metadataList = Lists.transform(adapters, IndexableAdapter::getMetadata); Metadata segmentMetadata = null; if (metricAggs != null) { @@ -177,21 +160,22 @@ public Metadata apply(IndexableAdapter input) final List dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size()); mergeCapabilities(adapters, mergedDimensions, metricsValueTypes, metricTypeNames, dimCapabilities); - final DimensionHandler[] handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities); + final Map handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities); final List mergers = new ArrayList<>(); for (int i = 0; i < mergedDimensions.size(); i++) { - mergers.add(handlers[i].makeMerger(indexSpec, segmentWriteOutMedium, dimCapabilities.get(i), progress)); + DimensionHandler handler = handlers.get(mergedDimensions.get(i)); + mergers.add(handler.makeMerger(indexSpec, segmentWriteOutMedium, dimCapabilities.get(i), progress)); } /************* Setup Dim Conversions **************/ progress.progress(); startTime = System.currentTimeMillis(); - writeDimValueAndSetupDimConversion(adapters, progress, mergedDimensions, mergers); + writeDimValuesAndSetupDimConversion(adapters, progress, mergedDimensions, mergers); log.info("Completed dim conversions in %,d millis.", System.currentTimeMillis() - startTime); /************* Walk through data sets, merge them, and write merged columns *************/ progress.progress(); - final Iterable theRows = makeRowIterable( + final TimeAndDimsIterator timeAndDimsIterator = makeMergedTimeAndDimsIterator( adapters, mergedDimensions, mergedMetrics, @@ -199,17 +183,19 @@ public Metadata apply(IndexableAdapter input) handlers, mergers ); + closer.register(timeAndDimsIterator); final GenericColumnSerializer timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec); - final ArrayList metWriters = setupMetricsWriters( - segmentWriteOutMedium, - mergedMetrics, - metricsValueTypes, - metricTypeNames, - indexSpec + final ArrayList metricWriters = + setupMetricsWriters(segmentWriteOutMedium, mergedMetrics, metricsValueTypes, metricTypeNames, indexSpec); + List rowNumConversions = mergeIndexesAndWriteColumns( + adapters, + progress, + timeAndDimsIterator, + timeWriter, + metricWriters, + mergers, + fillRowNumConversions ); - final List rowNumConversions = Lists.newArrayListWithCapacity(adapters.size()); - - mergeIndexesAndWriteColumns(adapters, progress, theRows, timeWriter, metWriters, rowNumConversions, mergers); /************ Create Inverted Indexes and Finalize Build Columns *************/ final String section = "build inverted index and columns"; @@ -221,7 +207,7 @@ public Metadata apply(IndexableAdapter input) mergedMetrics, metricsValueTypes, metricTypeNames, - metWriters, + metricWriters, indexSpec ); @@ -472,58 +458,87 @@ private void makeColumn( } } - private void mergeIndexesAndWriteColumns( + /** + * Returns rowNumConversions, if fillRowNumConversions argument is true + */ + @Nullable + private List mergeIndexesAndWriteColumns( final List adapters, final ProgressIndicator progress, - final Iterable theRows, + final TimeAndDimsIterator timeAndDimsIterator, final GenericColumnSerializer timeWriter, - final ArrayList metWriters, - final List rowNumConversions, - final List mergers + final ArrayList metricWriters, + final List mergers, + final boolean fillRowNumConversions ) throws IOException { final String section = "walk through and merge rows"; progress.startSection(section); long startTime = System.currentTimeMillis(); + List rowNumConversions = null; int rowCount = 0; - for (IndexableAdapter adapter : adapters) { - int[] arr = new int[adapter.getNumRows()]; - Arrays.fill(arr, INVALID_ROW); - rowNumConversions.add(IntBuffer.wrap(arr)); + if (fillRowNumConversions) { + rowNumConversions = new ArrayList<>(adapters.size()); + for (IndexableAdapter adapter : adapters) { + int[] arr = new int[adapter.getNumRows()]; + Arrays.fill(arr, INVALID_ROW); + rowNumConversions.add(IntBuffer.wrap(arr)); + } } long time = System.currentTimeMillis(); - for (Rowboat theRow : theRows) { + while (timeAndDimsIterator.moveToNext()) { progress.progress(); - timeWriter.serialize(theRow.getTimestamp()); + TimeAndDimsPointer timeAndDims = timeAndDimsIterator.getPointer(); + timeWriter.serialize(timeAndDims.timestampSelector); - final Object[] metrics = theRow.getMetrics(); - for (int i = 0; i < metrics.length; ++i) { - metWriters.get(i).serialize(metrics[i]); + for (int metricIndex = 0; metricIndex < timeAndDims.getNumMetrics(); metricIndex++) { + metricWriters.get(metricIndex).serialize(timeAndDims.getMetricSelector(metricIndex)); } - Object[] dims = theRow.getDims(); - for (int i = 0; i < dims.length; ++i) { - DimensionMerger merger = mergers.get(i); + for (int dimIndex = 0; dimIndex < timeAndDims.getNumDimensions(); dimIndex++) { + DimensionMerger merger = mergers.get(dimIndex); if (merger.canSkip()) { continue; } - merger.processMergedRow(dims[i]); + merger.processMergedRow(timeAndDims.getDimensionSelector(dimIndex)); } - Iterator> rowsIterator = theRow.getComprisedRows().int2ObjectEntrySet().fastIterator(); - while (rowsIterator.hasNext()) { - Int2ObjectMap.Entry comprisedRow = rowsIterator.next(); + if (timeAndDimsIterator instanceof RowCombiningTimeAndDimsIterator) { + RowCombiningTimeAndDimsIterator comprisedRows = (RowCombiningTimeAndDimsIterator) timeAndDimsIterator; + + for (int originalIteratorIndex = comprisedRows.nextCurrentlyCombinedOriginalIteratorIndex(0); + originalIteratorIndex >= 0; + originalIteratorIndex = + comprisedRows.nextCurrentlyCombinedOriginalIteratorIndex(originalIteratorIndex + 1)) { - final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getIntKey()); + IntBuffer conversionBuffer = rowNumConversions.get(originalIteratorIndex); + int minRowNum = comprisedRows.getMinCurrentlyCombinedRowNumByOriginalIteratorIndex(originalIteratorIndex); + int maxRowNum = comprisedRows.getMaxCurrentlyCombinedRowNumByOriginalIteratorIndex(originalIteratorIndex); - for (IntIterator setIterator = comprisedRow.getValue().iterator(); setIterator.hasNext(); /* NOP */) { - int rowNum = setIterator.nextInt(); - while (conversionBuffer.position() < rowNum) { - conversionBuffer.put(INVALID_ROW); + for (int rowNum = minRowNum; rowNum <= maxRowNum; rowNum++) { + while (conversionBuffer.position() < rowNum) { + conversionBuffer.put(INVALID_ROW); + } + conversionBuffer.put(rowCount); } - conversionBuffer.put(rowCount); + + } + + } else if (timeAndDimsIterator instanceof MergingRowIterator) { + RowPointer rowPointer = (RowPointer) timeAndDims; + IntBuffer conversionBuffer = rowNumConversions.get(rowPointer.getIndexNum()); + int rowNum = rowPointer.getRowNum(); + while (conversionBuffer.position() < rowNum) { + conversionBuffer.put(INVALID_ROW); + } + conversionBuffer.put(rowCount); + } else { + if (fillRowNumConversions) { + throw new IllegalStateException( + "Filling row num conversions is supported only with RowCombining and Merging iterators" + ); } } @@ -532,11 +547,14 @@ private void mergeIndexesAndWriteColumns( time = System.currentTimeMillis(); } } - for (IntBuffer rowNumConversion : rowNumConversions) { - rowNumConversion.rewind(); + if (rowNumConversions != null) { + for (IntBuffer rowNumConversion : rowNumConversions) { + rowNumConversion.rewind(); + } } log.info("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime); progress.stopSection(section); + return rowNumConversions; } private GenericColumnSerializer setupTimeWriter(SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec) @@ -664,7 +682,7 @@ static GenericColumnSerializer createFloatColumnSerializer( } } - private void writeDimValueAndSetupDimConversion( + private void writeDimValuesAndSetupDimConversion( final List indexes, final ProgressIndicator progress, final List mergedDimensions, @@ -675,7 +693,7 @@ private void writeDimValueAndSetupDimConversion( progress.startSection(section); for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) { - mergers.get(dimIndex).writeMergedValueMetadata(indexes); + mergers.get(dimIndex).writeMergedValueDictionary(indexes); } progress.stopSection(section); } @@ -691,20 +709,12 @@ private void mergeCapabilities( final Map capabilitiesMap = Maps.newHashMap(); for (IndexableAdapter adapter : adapters) { for (String dimension : adapter.getDimensionNames()) { - ColumnCapabilitiesImpl mergedCapabilities = capabilitiesMap.get(dimension); - if (mergedCapabilities == null) { - mergedCapabilities = new ColumnCapabilitiesImpl(); - mergedCapabilities.setType(null); - } - capabilitiesMap.put(dimension, mergedCapabilities.merge(adapter.getCapabilities(dimension))); + ColumnCapabilities capabilities = adapter.getCapabilities(dimension); + capabilitiesMap.computeIfAbsent(dimension, d -> new ColumnCapabilitiesImpl()).merge(capabilities); } for (String metric : adapter.getMetricNames()) { - ColumnCapabilitiesImpl mergedCapabilities = capabilitiesMap.get(metric); ColumnCapabilities capabilities = adapter.getCapabilities(metric); - if (mergedCapabilities == null) { - mergedCapabilities = new ColumnCapabilitiesImpl(); - } - capabilitiesMap.put(metric, mergedCapabilities.merge(capabilities)); + capabilitiesMap.computeIfAbsent(metric, m -> new ColumnCapabilitiesImpl()).merge(capabilities); metricsValueTypes.put(metric, capabilities.getType()); metricTypeNames.put(metric, adapter.getMetricType(metric)); } @@ -856,31 +866,8 @@ private File merge( final List mergedDimensions = IndexMerger.getMergedDimensions(indexes); - final List mergedMetrics = Lists.transform( - IndexMerger.mergeIndexed( - Lists.newArrayList( - FunctionalIterable - .create(indexes) - .transform( - new Function>() - { - @Override - public Iterable apply(@Nullable IndexableAdapter input) - { - return input.getMetricNames(); - } - } - ) - ) - ), - new Function() - { - @Override - public String apply(@Nullable String input) - { - return input; - } - } + final List mergedMetrics = IndexMerger.mergeIndexed( + indexes.stream().map(IndexableAdapter::getMetricNames).collect(Collectors.toList()) ); final AggregatorFactory[] sortedMetricAggs = new AggregatorFactory[mergedMetrics.size()]; @@ -916,34 +903,12 @@ public String apply(@Nullable String input) } } - Function>, Iterable> rowMergerFn = new Function>, Iterable>() - { - @Override - public Iterable apply( - @Nullable ArrayList> boats - ) - { - if (rollup) { - return CombiningIterable.create( - new MergeIterable<>(Comparators.naturalNullsFirst(), boats), - Comparators.naturalNullsFirst(), - new RowboatMergeFunction(sortedMetricAggs) - ); - } else { - return new MergeIterable( - new Ordering() - { - @Override - public int compare(Rowboat left, Rowboat right) - { - return Longs.compare(left.getTimestamp(), right.getTimestamp()); - } - }.nullsFirst(), - boats - ); - } - } - }; + Function, TimeAndDimsIterator> rowMergerFn; + if (rollup) { + rowMergerFn = rowIterators -> new RowCombiningTimeAndDimsIterator(rowIterators, sortedMetricAggs, mergedMetrics); + } else { + rowMergerFn = MergingRowIterator::new; + } return makeIndexFiles( indexes, @@ -953,6 +918,7 @@ public int compare(Rowboat left, Rowboat right) mergedDimensions, mergedMetrics, rowMergerFn, + true, indexSpec, segmentWriteOutMediumFactory ); @@ -982,15 +948,8 @@ public File convert( progress, Lists.newArrayList(adapter.getDimensionNames()), Lists.newArrayList(adapter.getMetricNames()), - new Function>, Iterable>() - { - @Nullable - @Override - public Iterable apply(ArrayList> input) - { - return input.get(0); - } - }, + Iterables::getOnlyElement, + false, indexSpec, segmentWriteOutMediumFactory ); @@ -1012,40 +971,9 @@ public File append( final List mergedDimensions = IndexMerger.getMergedDimensions(indexes); final List mergedMetrics = IndexMerger.mergeIndexed( - Lists.transform( - indexes, - new Function>() - { - @Override - public Iterable apply(@Nullable IndexableAdapter input) - { - return Iterables.transform( - input.getMetricNames(), - new Function() - { - @Override - public String apply(@Nullable String input) - { - return input; - } - } - ); - } - } - ) + indexes.stream().map(IndexableAdapter::getMetricNames).collect(Collectors.toList()) ); - Function>, Iterable> rowMergerFn = new Function>, Iterable>() - { - @Override - public Iterable apply( - @Nullable final ArrayList> boats - ) - { - return new MergeIterable<>(Comparators.naturalNullsFirst(), boats); - } - }; - return makeIndexFiles( indexes, aggregators, @@ -1053,123 +981,144 @@ public Iterable apply( new BaseProgressIndicator(), mergedDimensions, mergedMetrics, - rowMergerFn, + MergingRowIterator::new, + true, indexSpec, segmentWriteOutMediumFactory ); } - private DimensionHandler[] makeDimensionHandlers( + private Map makeDimensionHandlers( final List mergedDimensions, final List dimCapabilities ) { - final DimensionHandler[] handlers = new DimensionHandler[mergedDimensions.size()]; + Map handlers = new LinkedHashMap<>(); for (int i = 0; i < mergedDimensions.size(); i++) { ColumnCapabilities capabilities = dimCapabilities.get(i); String dimName = mergedDimensions.get(i); - handlers[i] = DimensionHandlerUtils.getHandlerFromCapabilities(dimName, capabilities, null); + DimensionHandler handler = DimensionHandlerUtils.getHandlerFromCapabilities(dimName, capabilities, null); + handlers.put(dimName, handler); } return handlers; } - private Iterable makeRowIterable( - List indexes, + private TimeAndDimsIterator makeMergedTimeAndDimsIterator( + final List indexes, final List mergedDimensions, final List mergedMetrics, - Function>, Iterable> rowMergerFn, - final DimensionHandler[] handlers, + final Function, TimeAndDimsIterator> rowMergerFn, + final Map handlers, final List mergers ) { - ArrayList> boats = Lists.newArrayListWithCapacity(indexes.size()); - + List perIndexRowIterators = Lists.newArrayListWithCapacity(indexes.size()); for (int i = 0; i < indexes.size(); ++i) { final IndexableAdapter adapter = indexes.get(i); - - final int[] dimLookup = getColumnIndexReorderingMap(adapter.getDimensionNames(), mergedDimensions); - final int[] metricLookup = getColumnIndexReorderingMap(adapter.getMetricNames(), mergedMetrics); - - Iterable target = indexes.get(i).getRows(); - if (dimLookup != null || metricLookup != null) { - // resize/reorder index table if needed - target = Iterables.transform( - target, - new Function() - { - @Override - public Rowboat apply(Rowboat input) - { - Object[] newDims; - if (dimLookup != null) { - newDims = new Object[mergedDimensions.size()]; - int j = 0; - for (Object dim : input.getDims()) { - newDims[dimLookup[j]] = dim; - j++; - } - } else { - // It's possible for getColumnIndexReorderingMap to return null when - // both column lists are identical. Copy the old array, no dimension reordering is needed. - newDims = input.getDims(); - } - - Object[] newMetrics = input.getMetrics(); - if (metricLookup != null) { - newMetrics = new Object[mergedMetrics.size()]; - int j = 0; - for (Object met : input.getMetrics()) { - newMetrics[metricLookup[j]] = met; - j++; - } - } - - return new Rowboat( - input.getTimestamp(), - newDims, - newMetrics, - input.getRowNum(), - handlers - ); - } - } + TransformableRowIterator target = adapter.getRows(); + if (!mergedDimensions.equals(adapter.getDimensionNames()) || !mergedMetrics.equals(adapter.getMetricNames())) { + target = makeRowIteratorWithReorderedColumns( + mergedDimensions, + mergedMetrics, + handlers, + adapter, + target ); } - boats.add( - new MMappedIndexRowIterable( - target, mergedDimensions, i, mergers - ) - ); + perIndexRowIterators.add(IndexMerger.toMergedIndexRowIterator(target, i, mergers)); } - - return rowMergerFn.apply(boats); + return rowMergerFn.apply(perIndexRowIterators); } - // If an adapter's column list differs from the merged column list across multiple indexes, - // return an array that maps the adapter's column orderings to the larger, merged column ordering - private int[] getColumnIndexReorderingMap(Indexed adapterColumnNames, List mergedColumnNames) + private TransformableRowIterator makeRowIteratorWithReorderedColumns( + List reorderedDimensions, + List reorderedMetrics, + Map originalHandlers, + IndexableAdapter originalAdapter, + TransformableRowIterator originalIterator + ) { - if (isSame(adapterColumnNames, mergedColumnNames)) { - return null; // no need to convert if column lists are identical - } - int[] dimLookup = new int[mergedColumnNames.size()]; - for (int i = 0; i < adapterColumnNames.size(); i++) { - dimLookup[i] = mergedColumnNames.indexOf(adapterColumnNames.get(i)); - } - return dimLookup; + RowPointer reorderedRowPointer = reorderRowPointerColumns( + reorderedDimensions, + reorderedMetrics, + originalHandlers, + originalAdapter, + originalIterator.getPointer() + ); + TimeAndDimsPointer reorderedMarkedRowPointer = reorderRowPointerColumns( + reorderedDimensions, + reorderedMetrics, + originalHandlers, + originalAdapter, + originalIterator.getMarkedPointer() + ); + return new ForwardingRowIterator(originalIterator) + { + @Override + public RowPointer getPointer() + { + return reorderedRowPointer; + } + + @Override + public TimeAndDimsPointer getMarkedPointer() + { + return reorderedMarkedRowPointer; + } + }; } - private boolean isSame(Indexed indexed, List values) + private static T reorderRowPointerColumns( + List reorderedDimensions, + List reorderedMetrics, + Map originalHandlers, + IndexableAdapter originalAdapter, + T originalRowPointer + ) { - if (indexed.size() != values.size()) { - return false; - } - for (int i = 0; i < indexed.size(); i++) { - if (!indexed.get(i).equals(values.get(i))) { - return false; - } + ColumnValueSelector[] reorderedDimensionSelectors = reorderedDimensions + .stream() + .map(dimName -> { + int dimIndex = originalAdapter.getDimensionNames().indexOf(dimName); + if (dimIndex >= 0) { + return originalRowPointer.getDimensionSelector(dimIndex); + } else { + return NilColumnValueSelector.instance(); + } + }) + .toArray(ColumnValueSelector[]::new); + List reorderedHandlers = + reorderedDimensions.stream().map(originalHandlers::get).collect(Collectors.toList()); + ColumnValueSelector[] reorderedMetricSelectors = reorderedMetrics + .stream() + .map(metricName -> { + int metricIndex = originalAdapter.getMetricNames().indexOf(metricName); + if (metricIndex >= 0) { + return originalRowPointer.getMetricSelector(metricIndex); + } else { + return NilColumnValueSelector.instance(); + } + }) + .toArray(ColumnValueSelector[]::new); + if (originalRowPointer instanceof RowPointer) { + //noinspection unchecked + return (T) new RowPointer( + originalRowPointer.timestampSelector, + reorderedDimensionSelectors, + reorderedHandlers, + reorderedMetricSelectors, + reorderedMetrics, + ((RowPointer) originalRowPointer).rowNumPointer + ); + } else { + //noinspection unchecked + return (T) new TimeAndDimsPointer( + originalRowPointer.timestampSelector, + reorderedDimensionSelectors, + reorderedHandlers, + reorderedMetricSelectors, + reorderedMetrics + ); } - return true; } - } diff --git a/processing/src/main/java/io/druid/segment/IndexableAdapter.java b/processing/src/main/java/io/druid/segment/IndexableAdapter.java index 56c026ba2b9f..8274ac5f89e3 100644 --- a/processing/src/main/java/io/druid/segment/IndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/IndexableAdapter.java @@ -24,7 +24,8 @@ import io.druid.segment.data.Indexed; import org.joda.time.Interval; -import java.util.Map; +import javax.annotation.Nullable; +import java.util.List; /** * An adapter to an index @@ -35,13 +36,14 @@ public interface IndexableAdapter int getNumRows(); - Indexed getDimensionNames(); + List getDimensionNames(); - Indexed getMetricNames(); + List getMetricNames(); - Indexed getDimValueLookup(String dimension); + @Nullable + > Indexed getDimValueLookup(String dimension); - Iterable getRows(); + TransformableRowIterator getRows(); BitmapValues getBitmapValues(String dimension, int dictId); @@ -50,6 +52,4 @@ public interface IndexableAdapter ColumnCapabilities getCapabilities(String column); Metadata getMetadata(); - - Map getDimensionHandlers(); } diff --git a/processing/src/main/java/io/druid/segment/LongColumnSerializer.java b/processing/src/main/java/io/druid/segment/LongColumnSerializer.java index bf8765680716..2c74fde458ff 100644 --- a/processing/src/main/java/io/druid/segment/LongColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/LongColumnSerializer.java @@ -81,10 +81,9 @@ public void open() throws IOException } @Override - public void serialize(Object obj) throws IOException + public void serialize(ColumnValueSelector selector) throws IOException { - long val = (obj == null) ? 0 : ((Number) obj).longValue(); - writer.add(val); + writer.add(selector.getLong()); } @Override diff --git a/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java index d9a71e015e4a..f5536dce0d8e 100644 --- a/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java +++ b/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java @@ -32,7 +32,6 @@ import io.druid.segment.data.CompressionStrategy; import io.druid.segment.writeout.SegmentWriteOutMedium; -import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; @@ -112,13 +111,13 @@ public void open() throws IOException } @Override - public void serialize(@Nullable Object obj) throws IOException + public void serialize(ColumnValueSelector selector) throws IOException { - if (obj == null) { + if (selector.isNull()) { nullRowsBitmap.add(rowCount); writer.add(0L); } else { - writer.add(((Number) obj).longValue()); + writer.add(selector.getLong()); } rowCount++; } diff --git a/processing/src/main/java/io/druid/segment/LongDimensionHandler.java b/processing/src/main/java/io/druid/segment/LongDimensionHandler.java index 3a27c0adf302..f91b9b1d12f0 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionHandler.java @@ -19,16 +19,25 @@ package io.druid.segment; -import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.GenericColumn; -import io.druid.segment.data.Indexed; +import io.druid.segment.selector.settable.SettableColumnValueSelector; +import io.druid.segment.selector.settable.SettableLongColumnValueSelector; import io.druid.segment.writeout.SegmentWriteOutMedium; -import java.io.Closeable; +import java.util.Comparator; public class LongDimensionHandler implements DimensionHandler { + private static Comparator LONG_COLUMN_COMPARATOR = (s1, s2) -> { + if (s1.isNull()) { + return s2.isNull() ? 0 : -1; + } else if (s2.isNull()) { + return 1; + } else { + return Long.compare(s1.getLong(), s2.getLong()); + } + }; + private final String dimensionName; public LongDimensionHandler(String dimensionName) @@ -49,7 +58,7 @@ public DimensionIndexer makeIndexer() } @Override - public DimensionMergerV9 makeMerger( + public DimensionMergerV9 makeMerger( IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -70,35 +79,14 @@ public int getLengthOfEncodedKeyComponent(Long dimVals) } @Override - public int compareSortedEncodedKeyComponents(Long lhs, Long rhs) - { - return lhs.compareTo(rhs); - } - - @Override - public void validateSortedEncodedKeyComponents( - Long lhs, Long rhs, Indexed lhsEncodings, Indexed rhsEncodings - ) throws SegmentValidationException - { - if (!lhs.equals(rhs)) { - throw new SegmentValidationException( - "Dim [%s] value not equal. Expected [%s] found [%s]", - dimensionName, - lhs, - rhs - ); - } - } - - @Override - public Closeable getSubColumn(Column column) + public Comparator getEncodedValueSelectorComparator() { - return column.getGenericColumn(); + return LONG_COLUMN_COMPARATOR; } @Override - public Long getEncodedKeyComponentFromColumn(Closeable column, int currRow) + public SettableColumnValueSelector makeNewSettableEncodedValueSelector() { - return ((GenericColumn) column).getLongSingleValueRow(currRow); + return new SettableLongColumnValueSelector(); } } diff --git a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java index 187c3fe2d291..387fdf4d357f 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java @@ -27,7 +27,7 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; -import io.druid.segment.incremental.TimeAndDimsHolder; +import io.druid.segment.incremental.IncrementalIndexRowHolder; import javax.annotation.Nullable; import java.util.Comparator; @@ -83,7 +83,7 @@ public int getCardinality() @Override public DimensionSelector makeDimensionSelector( DimensionSpec spec, - TimeAndDimsHolder currEntry, + IncrementalIndexRowHolder currEntry, IncrementalIndex.DimensionDesc desc ) { @@ -92,7 +92,7 @@ public DimensionSelector makeDimensionSelector( @Override public ColumnValueSelector makeColumnValueSelector( - TimeAndDimsHolder currEntry, + IncrementalIndexRowHolder currEntry, IncrementalIndex.DimensionDesc desc ) { @@ -169,9 +169,9 @@ public Object convertUnsortedEncodedKeyComponentToActualArrayOrList(Long key, bo } @Override - public Long convertUnsortedEncodedKeyComponentToSortedEncodedKeyComponent(Long key) + public ColumnValueSelector convertUnsortedValuesToSorted(ColumnValueSelector selectorWithUnsortedValues) { - return key; + return selectorWithUnsortedValues; } @Override diff --git a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java index e2bd12cca80b..4f3aa7406db4 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java @@ -19,77 +19,23 @@ package io.druid.segment; -import com.google.common.base.Throwables; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; import io.druid.segment.serde.ColumnPartSerde; import io.druid.segment.writeout.SegmentWriteOutMedium; -import java.io.IOException; -import java.nio.IntBuffer; -import java.util.List; - -public class LongDimensionMergerV9 implements DimensionMergerV9 +public class LongDimensionMergerV9 extends NumericDimensionMergerV9 { - protected String dimensionName; - protected final IndexSpec indexSpec; - protected GenericColumnSerializer serializer; - - LongDimensionMergerV9( - String dimensionName, - IndexSpec indexSpec, - SegmentWriteOutMedium segmentWriteOutMedium - ) - { - this.dimensionName = dimensionName; - this.indexSpec = indexSpec; - try { - setupEncodedValueWriter(segmentWriteOutMedium); - } - catch (IOException ioe) { - Throwables.propagate(ioe); - } - } - - protected void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException - { - this.serializer = IndexMergerV9.createLongColumnSerializer( - segmentWriteOutMedium, - dimensionName, - indexSpec - ); - serializer.open(); - } - - @Override - public void writeMergedValueMetadata(List adapters) - { - // longs have no additional metadata - } - - @Override - public Long convertSegmentRowValuesToMergedRowValues(Long segmentRow, int segmentIndexNumber) - { - return segmentRow; - } - - @Override - public void processMergedRow(Long rowValues) throws IOException - { - serializer.serialize(rowValues); - } - - @Override - public void writeIndexes(List segmentRowNumConversions) + LongDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) { - // longs have no indices to write + super(dimensionName, indexSpec, segmentWriteOutMedium); } @Override - public boolean canSkip() + GenericColumnSerializer setupEncodedValueWriter() { - return false; + return IndexMergerV9.createLongColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec); } @Override diff --git a/processing/src/main/java/io/druid/segment/MergingRowIterator.java b/processing/src/main/java/io/druid/segment/MergingRowIterator.java new file mode 100644 index 000000000000..c24a1babcf18 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/MergingRowIterator.java @@ -0,0 +1,273 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import io.druid.java.util.common.io.Closer; +import it.unimi.dsi.fastutil.objects.ObjectHeaps; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Comparator; +import java.util.List; +import java.util.stream.IntStream; + +/** + * MergingRowIterator sort-merges rows of several {@link RowIterator}s, assuming that each of them is already sorted + * (i. e. as {@link RowIterator#moveToNext()} is called, the pointer returned from {@link RowIterator#getPointer()} is + * "greater" than the previous, in terms of {@link TimeAndDimsPointer#compareTo}). Equivalent points from different + * input iterators are _not_ deduplicated. + * + * Conceptually MergingRowIterator is an equivalent to {@link com.google.common.collect.Iterators#mergeSorted}, but for + * {@link RowIterator}s rather than simple {@link java.util.Iterator}s. + * + * Implementation detail: this class uses binary heap priority queue algorithm to sort pointers, but it also memoizes + * and keeps some extra info along the heap slots (see javadoc of {@link #equalToChild} field), that is essential for + * impelementing {@link #hasTimeAndDimsChangedSinceMark()}, while {@link TransformableRowIterator#getPointer()} of the + * input iterators are mutable pointers, and you cannot "look back" to compare with some past point. + */ +final class MergingRowIterator implements RowIterator +{ + private static final Comparator ROW_ITERATOR_COMPARATOR = + Comparator.comparing(RowIterator::getPointer); + + /** Used to close {@link #originalIterators} */ + private final Closer closer = Closer.create(); + + private final TransformableRowIterator[] originalIterators; + + /** Binary heap (priority queue) */ + private final RowIterator[] pQueue; + private int pQueueSize; + + /** + * Boolean flags corresponding to the elements of {@link #pQueue} binary heap, signifying if the element is equal + * to one or both children in the binary heap. For leaf elements (i. e. no children), the value should be false. + */ + private final boolean[] equalToChild; + + /** true while {@link #moveToNext()} is not called yet. */ + private boolean first = true; + + /** + * True by default, so that if {@link #mark()} is never called, the extra work to compute the value for this field is + * never done. + */ + private boolean changedSinceMark = true; + + /** This field is needed for some optimization, see the comment where it is used. */ + @Nullable + private RowIterator lastMarkedHead = null; + + MergingRowIterator(List iterators) + { + iterators.forEach(closer::register); + originalIterators = new TransformableRowIterator[iterators.size()]; + pQueue = IntStream + .range(0, iterators.size()) + .filter(indexNum -> iterators.get(indexNum).moveToNext()) + .mapToObj(indexNum -> { + TransformableRowIterator rowIterator = iterators.get(indexNum); + // Can call rowIterator.getPointer() only here, after moveToNext() returned true on the filter() step + rowIterator.getPointer().setIndexNum(indexNum); + originalIterators[indexNum] = rowIterator; + return rowIterator; + }) + .toArray(RowIterator[]::new); + equalToChild = new boolean[pQueue.length]; + heapify(); + initEqualToChildStates(); + } + + @Nullable + TransformableRowIterator getOriginalIterator(int indexNum) + { + return originalIterators[indexNum]; + } + + @Override + public RowPointer getPointer() + { + // Current MergingRowIterator's pointer is the pointer of the head of the priority queue. + return pQueue[0].getPointer(); + } + + @Override + public boolean moveToNext() + { + if (pQueueSize == 0) { + if (first) { + first = false; + return false; + } + throw new IllegalStateException("Don't call moveToNext() after it returned false once"); + } + if (first) { + first = false; + return true; + } + + RowIterator head = pQueue[0]; + if (!changedSinceMark) { + // lastMarkedHead field allows small optimization: avoiding many re-marks of the rows of the head iterator, + // if it has many elements with equal dimensions. + //noinspection ObjectEquality: checking specifically if lastMarkedHead and head is the same object + if (lastMarkedHead != head) { + head.mark(); + lastMarkedHead = head; + } + } + boolean headUsedToBeEqualToChild = equalToChild[0]; + if (head.moveToNext()) { + if (sinkHeap(0) == 0) { // The head iterator didn't change + if (!changedSinceMark && head.hasTimeAndDimsChangedSinceMark()) { + changedSinceMark = true; + } + } else { // The head iterator changed + // If the head iterator changed, the changedSinceMark property could still be "unchanged", if there were several + // iterators pointing to equal "time and dims", that is what the following line checks: + changedSinceMark |= !headUsedToBeEqualToChild; + } + return true; + } else { + pQueueSize--; + if (pQueueSize > 0) { + pQueue[0] = pQueue[pQueueSize]; + pQueue[pQueueSize] = null; + + // The head iterator is going to change, so the changedSinceMark property could still be "unchanged", if there + // were several iterators pointing to equal "time and dims", that is what the following line checks: + changedSinceMark |= !headUsedToBeEqualToChild; + + int parentOfLast = (pQueueSize - 1) >> 1; + // This sinkHeap() call is guaranteed to not move any heap elements, but it is used as a shortcut to fix the + // equalToChild[parentOfLast] status, as a side-effect. equalToChild[parentOfLast] could have changed, e. g. if + // the last element was equal to parentOfLast, and parentOfLast had only one child, or the other child is + // different. In this case, equalToChild[parentOfLast] is going to be changed from true to false, because the + // last element is now moved to the head. equalToChild[parentOfLast] must have correct value before the + // sinkHeap(0) call a few lines below, because it's an assumed invariant of sinkHeap(). + sinkHeap(parentOfLast); + sinkHeap(0); + return true; + } else { + // Don't clear pQueue[0], to conform to RowIterator.getPointer() specification. + // Don't care about changedSinceMark, because according to the RowIterator specification the behaviour of + // hasTimeAndDimsChangedSinceMark() is undefined now. + return false; + } + } + } + + @Override + public void mark() + { + changedSinceMark = false; + lastMarkedHead = null; + } + + @Override + public boolean hasTimeAndDimsChangedSinceMark() + { + return changedSinceMark; + } + + /** + * Sinks (pushes down) the iterator at the given index i, because it's value just changed due to {@link + * RowIterator#moveToNext()} call. + * + * This method implementation is originally based on {@link ObjectHeaps#downHeap} implementation, plus it updates + * {@link #equalToChild} states along the way. + */ + private int sinkHeap(int i) + { + final RowIterator iteratorToSink = pQueue[i]; + while (true) { + int left = (i << 1) + 1; + if (left >= pQueueSize) { + break; + } + int child = left; + RowIterator childIterator = pQueue[left]; + final int right = left + 1; + // Setting childrenDiff to non-zero initially, so that if there is just one child (right == pQueueSize), + // the expression `childrenDiff == 0` below is false. + int childrenDiff = -1; // any non-zero number + if (right < pQueueSize) { + RowIterator rightChildIterator = pQueue[right]; + childrenDiff = rightChildIterator.getPointer().compareTo(childIterator.getPointer()); + if (childrenDiff < 0) { + child = right; + childIterator = rightChildIterator; + } + } + int parentDiff = iteratorToSink.getPointer().compareTo(childIterator.getPointer()); + if (parentDiff == 0) { + equalToChild[i] = true; + pQueue[i] = iteratorToSink; + return i; + } + if (parentDiff < 0) { + equalToChild[i] = false; + pQueue[i] = iteratorToSink; + return i; + } + equalToChild[i] = childrenDiff == 0 || equalToChild[child]; + pQueue[i] = childIterator; + i = child; + } + equalToChild[i] = false; // Since we exited from the above loop, there are no more children + pQueue[i] = iteratorToSink; + return i; + } + + private void heapify() + { + pQueueSize = 0; + while (pQueueSize < pQueue.length) { + pQueueSize++; + ObjectHeaps.upHeap(pQueue, pQueueSize, pQueueSize - 1, ROW_ITERATOR_COMPARATOR); + } + } + + private void initEqualToChildStates() + { + for (int i = 0; i < pQueueSize; i++) { + int left = (i << 1) + 1; + int right = left + 1; + equalToChild[i] = (left < pQueueSize && iteratorsEqual(i, left)) || + (right < pQueueSize && iteratorsEqual(i, right)); + } + } + + private boolean iteratorsEqual(int i1, int i2) + { + return pQueue[i1].getPointer().compareTo(pQueue[i2].getPointer()) == 0; + } + + @Override + public void close() + { + try { + closer.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/NullDimensionSelector.java b/processing/src/main/java/io/druid/segment/NullDimensionSelector.java index 56ad73fcae1e..b65f0418fe73 100644 --- a/processing/src/main/java/io/druid/segment/NullDimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/NullDimensionSelector.java @@ -81,6 +81,7 @@ public int getValueCardinality() } @Override + @Nullable public String lookupName(int id) { assert id == 0 : "id = " + id; diff --git a/processing/src/main/java/io/druid/segment/NumericDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/NumericDimensionMergerV9.java new file mode 100644 index 000000000000..bb9c03b78df0 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/NumericDimensionMergerV9.java @@ -0,0 +1,93 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import io.druid.segment.writeout.SegmentWriteOutMedium; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.IntBuffer; +import java.util.List; + +/** + * Common base of {@link LongDimensionMergerV9}, {@link DoubleDimensionMergerV9} and {@link FloatDimensionMergerV9}. + */ +public abstract class NumericDimensionMergerV9 implements DimensionMergerV9 +{ + protected final String dimensionName; + protected final IndexSpec indexSpec; + protected final SegmentWriteOutMedium segmentWriteOutMedium; + + protected final GenericColumnSerializer serializer; + + NumericDimensionMergerV9( + String dimensionName, + IndexSpec indexSpec, + SegmentWriteOutMedium segmentWriteOutMedium + ) + { + this.dimensionName = dimensionName; + this.indexSpec = indexSpec; + this.segmentWriteOutMedium = segmentWriteOutMedium; + + try { + serializer = setupEncodedValueWriter(); + serializer.open(); + } + catch (IOException ioe) { + throw new RuntimeException(ioe); + } + } + + abstract GenericColumnSerializer setupEncodedValueWriter(); + + @Override + public final void writeMergedValueDictionary(List adapters) + { + // numeric values have no additional metadata + } + + @Override + public final ColumnValueSelector convertSortedSegmentRowValuesToMergedRowValues( + int segmentIndex, + ColumnValueSelector source + ) + { + return source; + } + + @Override + public final void processMergedRow(ColumnValueSelector selector) throws IOException + { + serializer.serialize(selector); + } + + @Override + public final void writeIndexes(@Nullable List segmentRowNumConversions) + { + // numeric values have no indices to write + } + + @Override + public final boolean canSkip() + { + return false; + } +} diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index 431c050bde8a..16ba427e9d0f 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -20,39 +20,35 @@ package io.druid.segment; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.Lists; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.logger.Logger; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.column.BaseColumn; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ComplexColumn; import io.druid.segment.column.DictionaryEncodedColumn; -import io.druid.segment.column.GenericColumn; -import io.druid.segment.column.DoublesColumn; -import io.druid.segment.column.FloatsColumn; -import io.druid.segment.column.LongsColumn; import io.druid.segment.column.ValueType; import io.druid.segment.data.BitmapValues; import io.druid.segment.data.ImmutableBitmapValues; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedIterable; -import io.druid.segment.data.ListIndexed; +import io.druid.segment.selector.settable.SettableColumnValueSelector; +import io.druid.segment.selector.settable.SettableLongColumnValueSelector; import org.joda.time.Interval; -import java.io.Closeable; -import java.util.Collection; +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.NoSuchElementException; import java.util.Set; /** @@ -60,9 +56,10 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter { private static final Logger log = new Logger(QueryableIndexIndexableAdapter.class); + private final int numRows; private final QueryableIndex input; - private final List availableDimensions; + private final ImmutableList availableDimensions; private final Metadata metadata; public QueryableIndexIndexableAdapter(QueryableIndex input) @@ -73,7 +70,7 @@ public QueryableIndexIndexableAdapter(QueryableIndex input) // It appears possible that the dimensions have some columns listed which do not have a DictionaryEncodedColumn // This breaks current logic, but should be fine going forward. This is a work-around to make things work // in the current state. This code shouldn't be needed once github tracker issue #55 is finished. - this.availableDimensions = Lists.newArrayList(); + ImmutableList.Builder availableDimensions = ImmutableList.builder(); for (String dim : input.getAvailableDimensions()) { final Column col = input.getColumn(dim); @@ -86,6 +83,7 @@ public QueryableIndexIndexableAdapter(QueryableIndex input) availableDimensions.add(dim); } } + this.availableDimensions = availableDimensions.build(); this.metadata = input.getMetadata(); } @@ -103,25 +101,22 @@ public int getNumRows() } @Override - public Indexed getDimensionNames() + public List getDimensionNames() { - return new ListIndexed<>(availableDimensions, String.class); + return availableDimensions; } @Override - public Indexed getMetricNames() + public List getMetricNames() { final Set columns = Sets.newLinkedHashSet(input.getColumnNames()); final HashSet dimensions = Sets.newHashSet(getDimensionNames()); - - return new ListIndexed<>( - Lists.newArrayList(Sets.difference(columns, dimensions)), - String.class - ); + return ImmutableList.copyOf(Sets.difference(columns, dimensions)); } + @Nullable @Override - public Indexed getDimValueLookup(String dimension) + public > Indexed getDimValueLookup(String dimension) { final Column column = input.getColumn(dimension); @@ -129,18 +124,18 @@ public Indexed getDimValueLookup(String dimension) return null; } - final DictionaryEncodedColumn dict = column.getDictionaryEncoding(); + final DictionaryEncodedColumn dict = column.getDictionaryEncoding(); if (dict == null) { return null; } - return new Indexed() + return new Indexed() { @Override - public Class getClazz() + public Class getClazz() { - return Comparable.class; + return dict.getClazz(); } @Override @@ -150,19 +145,19 @@ public int size() } @Override - public Comparable get(int index) + public T get(int index) { return dict.lookupName(index); } @Override - public int indexOf(Comparable value) + public int indexOf(T value) { return dict.lookupId(value); } @Override - public Iterator iterator() + public Iterator iterator() { return IndexedIterable.create(this).iterator(); } @@ -176,130 +171,200 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) } @Override - public Iterable getRows() + public RowIteratorImpl getRows() { - return new Iterable() + return new RowIteratorImpl(); + } + + /** + * On {@link #moveToNext()} and {@link #mark()}, this class copies all column values into a set of {@link + * SettableColumnValueSelector} instances. Alternative approach was to save only offset in column and use the same + * column value selectors as in {@link QueryableIndexStorageAdapter}. The approach with "caching" in {@link + * SettableColumnValueSelector}s is chosen for two reasons: + * 1) Avoid re-reading column values from serialized format multiple times (because they are accessed multiple times) + * For comparison, it's not a factor for {@link QueryableIndexStorageAdapter} because during query processing, + * column values are usually accessed just once per offset, if aggregator or query runner are written sanely. + * Avoiding re-reads is especially important for object columns, because object deserialization is potentially + * expensive. + * 2) {@link #mark()} is a "lookbehind" style functionality, in compressed columnar format, that would cause + * repetitive excessive decompressions on the block boundaries. E. g. see {@link + * io.druid.segment.data.BlockLayoutColumnarDoublesSupplier} and similar classes. Some special support for + * "lookbehind" could be added to these classes, but it's significant extra complexity. + */ + class RowIteratorImpl implements TransformableRowIterator + { + private final Closer closer = Closer.create(); + private final Map columnCache = new HashMap<>(); + + private final SimpleAscendingOffset offset = new SimpleAscendingOffset(numRows); + private final int maxValidOffset = numRows - 1; + + private final ColumnValueSelector offsetTimestampSelector; + private final ColumnValueSelector[] offsetDimensionValueSelectors; + private final ColumnValueSelector[] offsetMetricSelectors; + + private final SettableLongColumnValueSelector rowTimestampSelector = new SettableLongColumnValueSelector(); + private final SettableColumnValueSelector[] rowDimensionValueSelectors; + private final SettableColumnValueSelector[] rowMetricSelectors; + private final RowPointer rowPointer; + + private final SettableLongColumnValueSelector markedTimestampSelector = new SettableLongColumnValueSelector(); + private final SettableColumnValueSelector[] markedDimensionValueSelectors; + private final SettableColumnValueSelector[] markedMetricSelectors; + private final TimeAndDimsPointer markedRowPointer; + + boolean first = true; + int memoizedOffset = -1; + + RowIteratorImpl() { - @Override - public Iterator iterator() - { - return new Iterator() - { - final GenericColumn timestamps = input.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn(); - final Closeable[] metrics; - final Closeable[] columns; - final Closer closer = Closer.create(); - - final int numMetrics = getMetricNames().size(); - - final DimensionHandler[] handlers = new DimensionHandler[availableDimensions.size()]; - Collection handlerSet = input.getDimensionHandlers().values(); - - int currRow = 0; - boolean done = false; - - { - closer.register(timestamps); - - handlerSet.toArray(handlers); - this.columns = FluentIterable - .from(handlerSet) - .transform( - new Function() - { - @Override - public Closeable apply(DimensionHandler handler) - { - Column column = input.getColumn(handler.getDimensionName()); - return handler.getSubColumn(column); - } - } - ).toArray(Closeable.class); - for (Closeable column : columns) { - closer.register(column); - } - - final Indexed availableMetrics = getMetricNames(); - metrics = new Closeable[availableMetrics.size()]; - for (int i = 0; i < metrics.length; ++i) { - final Column column = input.getColumn(availableMetrics.get(i)); - final ValueType type = column.getCapabilities().getType(); - switch (type) { - case FLOAT: - case LONG: - case DOUBLE: - metrics[i] = column.getGenericColumn(); - break; - case COMPLEX: - metrics[i] = column.getComplexColumn(); - break; - default: - throw new ISE("Cannot handle type[%s]", type); - } - } - for (Closeable metricColumn : metrics) { - closer.register(metricColumn); - } - } - - @Override - public boolean hasNext() - { - final boolean hasNext = currRow < numRows; - if (!hasNext && !done) { - CloseQuietly.close(closer); - done = true; - } - return hasNext; - } - - @Override - public Rowboat next() - { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - final Object[] dims = new Object[columns.length]; - int dimIndex = 0; - for (final Closeable column : columns) { - dims[dimIndex] = handlers[dimIndex].getEncodedKeyComponentFromColumn(column, currRow); - dimIndex++; - } - - Object[] metricArray = new Object[numMetrics]; - for (int i = 0; i < metricArray.length; ++i) { - if (metrics[i] instanceof FloatsColumn) { - GenericColumn genericColumn = (GenericColumn) metrics[i]; - metricArray[i] = genericColumn.isNull(currRow) ? null : genericColumn.getFloatSingleValueRow(currRow); - } else if (metrics[i] instanceof DoublesColumn) { - GenericColumn genericColumn = (GenericColumn) metrics[i]; - metricArray[i] = genericColumn.isNull(currRow) ? null : genericColumn.getDoubleSingleValueRow(currRow); - } else if (metrics[i] instanceof LongsColumn) { - GenericColumn genericColumn = (GenericColumn) metrics[i]; - metricArray[i] = genericColumn.isNull(currRow) ? null : genericColumn.getLongSingleValueRow(currRow); - } else if (metrics[i] instanceof ComplexColumn) { - metricArray[i] = ((ComplexColumn) metrics[i]).getRowValue(currRow); - } - } - - final Rowboat retVal = new Rowboat( - timestamps.getLongSingleValueRow(currRow), dims, metricArray, currRow, handlers - ); - - ++currRow; - - return retVal; - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }; + final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( + input, + VirtualColumns.EMPTY, + false, + closer, + offset, + columnCache + ); + + offsetTimestampSelector = columnSelectorFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME); + + final List dimensionHandlers = new ArrayList<>(input.getDimensionHandlers().values()); + + offsetDimensionValueSelectors = dimensionHandlers + .stream() + .map(DimensionHandler::getDimensionName) + .map(columnSelectorFactory::makeColumnValueSelector) + .toArray(ColumnValueSelector[]::new); + + List metricNames = getMetricNames(); + offsetMetricSelectors = + metricNames.stream().map(columnSelectorFactory::makeColumnValueSelector).toArray(ColumnValueSelector[]::new); + + rowDimensionValueSelectors = dimensionHandlers + .stream() + .map(DimensionHandler::makeNewSettableEncodedValueSelector) + .toArray(SettableColumnValueSelector[]::new); + rowMetricSelectors = metricNames + .stream() + .map(metric -> input.getColumn(metric).makeSettableColumnValueSelector()) + .toArray(SettableColumnValueSelector[]::new); + + rowPointer = new RowPointer( + rowTimestampSelector, + rowDimensionValueSelectors, + dimensionHandlers, + rowMetricSelectors, + metricNames, + offset::getOffset + ); + + + markedDimensionValueSelectors = dimensionHandlers + .stream() + .map(DimensionHandler::makeNewSettableEncodedValueSelector) + .toArray(SettableColumnValueSelector[]::new); + markedMetricSelectors = metricNames + .stream() + .map(metric -> input.getColumn(metric).makeSettableColumnValueSelector()) + .toArray(SettableColumnValueSelector[]::new); + markedRowPointer = new TimeAndDimsPointer( + markedTimestampSelector, + markedDimensionValueSelectors, + dimensionHandlers, + markedMetricSelectors, + metricNames + ); + } + + @Override + public TimeAndDimsPointer getMarkedPointer() + { + return markedRowPointer; + } + + /** + * When a segment is produced using "rollup", each row is guaranteed to have different dimensions, so this method + * could be optimized to have just "return true;" body. + * TODO record in the segment metadata if each row has different dims or not, to be able to apply this optimization. + */ + @Override + public boolean hasTimeAndDimsChangedSinceMark() + { + return markedRowPointer.compareTo(rowPointer) != 0; + } + + @Override + public void close() + { + CloseQuietly.close(closer); + } + + @Override + public RowPointer getPointer() + { + return rowPointer; + } + + @Override + public boolean moveToNext() + { + if (first) { + first = false; + if (offset.withinBounds()) { + setRowPointerValues(); + return true; + } else { + return false; + } + } else { + if (offset.getOffset() < maxValidOffset) { + offset.increment(); + setRowPointerValues(); + return true; + } else { + // Don't update rowPointer's values here, to conform to the RowIterator.getPointer() specification. + return false; + } } - }; + } + + private void setRowPointerValues() + { + rowTimestampSelector.setValue(offsetTimestampSelector.getLong()); + for (int i = 0; i < offsetDimensionValueSelectors.length; i++) { + rowDimensionValueSelectors[i].setValueFrom(offsetDimensionValueSelectors[i]); + } + for (int i = 0; i < offsetMetricSelectors.length; i++) { + rowMetricSelectors[i].setValueFrom(offsetMetricSelectors[i]); + } + } + + @Override + public void mark() + { + markedTimestampSelector.setValue(rowTimestampSelector.getLong()); + for (int i = 0; i < rowDimensionValueSelectors.length; i++) { + markedDimensionValueSelectors[i].setValueFrom(rowDimensionValueSelectors[i]); + } + for (int i = 0; i < rowMetricSelectors.length; i++) { + markedMetricSelectors[i].setValueFrom(rowMetricSelectors[i]); + } + } + + /** + * Used in {@link RowFilteringIndexAdapter} + */ + void memoizeOffset() + { + memoizedOffset = offset.getOffset(); + } + + void resetToMemoizedOffset() + { + offset.setCurrentOffset(memoizedOffset); + setRowPointerValues(); + } } @Override @@ -373,10 +438,4 @@ public Metadata getMetadata() { return metadata; } - - @Override - public Map getDimensionHandlers() - { - return input.getDimensionHandlers(); - } } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 6d3bcc373753..4faa794309f2 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -53,6 +53,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -89,7 +90,8 @@ public Indexed getAvailableDimensions() @Override public Iterable getAvailableMetrics() { - return Sets.difference(Sets.newHashSet(index.getColumnNames()), Sets.newHashSet(index.getAvailableDimensions())); + HashSet columnNames = Sets.newHashSet(index.getColumnNames()); + return Sets.difference(columnNames, Sets.newHashSet(index.getAvailableDimensions())); } @Override @@ -235,7 +237,7 @@ public Sequence makeCursors( int preFilteredRows = totalRows; if (filter == null) { preFilters = Collections.emptyList(); - offset = new NoFilterOffset(0, totalRows, descending); + offset = descending ? new SimpleDescendingOffset(totalRows) : new SimpleAscendingOffset(totalRows); } else { preFilters = new ArrayList<>(); @@ -258,7 +260,7 @@ public Sequence makeCursors( } if (preFilters.size() == 0) { - offset = new NoFilterOffset(0, index.getNumRows(), descending); + offset = descending ? new SimpleDescendingOffset(totalRows) : new SimpleAscendingOffset(totalRows); } else { if (queryMetrics != null) { BitmapResultFactory bitmapResultFactory = @@ -333,6 +335,7 @@ private static class CursorSequenceBuilder private final long minDataTimestamp; private final long maxDataTimestamp; private final boolean descending; + @Nullable private final Filter postFilter; private final ColumnSelectorBitmapIndexSelector bitmapIndexSelector; @@ -345,7 +348,7 @@ public CursorSequenceBuilder( long minDataTimestamp, long maxDataTimestamp, boolean descending, - Filter postFilter, + @Nullable Filter postFilter, ColumnSelectorBitmapIndexSelector bitmapIndexSelector ) { @@ -548,7 +551,8 @@ public abstract static class TimestampCheckingOffset extends Offset this.baseOffset = baseOffset; this.timestamps = timestamps; this.timeLimit = timeLimit; - // checks if all the values are within the Threshold specified, skips timestamp lookups and checks if all values are within threshold. + // checks if all the values are within the Threshold specified, skips timestamp lookups and checks if all values + // are within threshold. this.allWithinThreshold = allWithinThreshold; } diff --git a/processing/src/main/java/io/druid/segment/RowCombiningTimeAndDimsIterator.java b/processing/src/main/java/io/druid/segment/RowCombiningTimeAndDimsIterator.java new file mode 100644 index 000000000000..a66cb7a3b0f8 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/RowCombiningTimeAndDimsIterator.java @@ -0,0 +1,303 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import io.druid.query.aggregation.AggregateCombiner; +import io.druid.query.aggregation.AggregatorFactory; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.BitSet; +import java.util.List; +import java.util.Objects; + +/** + * RowCombiningTimeAndDimsIterator takes some {@link RowIterator}s, assuming that they are "sorted" (see javadoc of + * {@link MergingRowIterator} for the definition), merges the points as {@link MergingRowIterator}, and combines + * all "equal" points (points which have the same time and dimension values) into one, using the provided metric + * aggregator factories. + */ +final class RowCombiningTimeAndDimsIterator implements TimeAndDimsIterator +{ + private static final int MIN_CURRENTLY_COMBINED_ROW_NUM_UNSET_VALUE = -1; + + private final MergingRowIterator mergingIterator; + + /** + * Those pointers are used as {@link #currentTimeAndDimsPointer} (and therefore returned from {@link #getPointer()}), + * until there is just one point in the "equivalence class" to be combined. It's an optimization: alternative was to + * start to "combine" right away for each point, and thus use only {@link + * #combinedTimeAndDimsPointersByOriginalIteratorIndex}. This optimization aims to reduce data movements from pointers + * of original iterators to {@link #combinedTimeAndDimsPointersByOriginalIteratorIndex} on each iteration. + * + * @see #soleCurrentPointSourceOriginalIteratorIndex + */ + private final TimeAndDimsPointer[] markedRowPointersOfOriginalIterators; + + private final AggregateCombiner[] combinedMetricSelectors; + + private final List combinedMetricNames; + + /** + * We preserve as many "combined time and dims pointers" as there were original iterators. Each of them is a composite + * of time and dimension selectors from the original iterator by the corresponding index, and the same metric + * selectors ({@link #combinedMetricSelectors}). It allows to be allocation-free during iteration, and also to reduce + * the number of field writes during iteration. + */ + private final TimeAndDimsPointer[] combinedTimeAndDimsPointersByOriginalIteratorIndex; + + /** + * This bitset has set bits that correspond to the indexes of the original iterators, that participate the current + * combination of all points from the current "equivalence class", resulting in the current {@link #getPointer()} + * point. + * + * If there are less than 64 iterators combined, this field could be optimized to be just a single primitive long. + * This optimization could be done in the future. + */ + private final BitSet indexesOfCurrentlyCombinedOriginalIterators = new BitSet(); + + /** + * This field and {@link #maxCurrentlyCombinedRowNumByOriginalIteratorIndex} designate "row num range" in each + * original iterator, points from which are currently combined (see {@link + * #indexesOfCurrentlyCombinedOriginalIterators}). It could have been a single row number, if original iterators were + * guaranteed to have no duplicate rows themselves, but they are not. + */ + private final int[] minCurrentlyCombinedRowNumByOriginalIteratorIndex; + private final int[] maxCurrentlyCombinedRowNumByOriginalIteratorIndex; + + @Nullable + private TimeAndDimsPointer currentTimeAndDimsPointer; + + /** + * If soleCurrentPointSourceOriginalIteratorIndex >= 0, it means that no combines are done yet at the current point, + * {@link #currentTimeAndDimsPointer} is one of {@link #markedRowPointersOfOriginalIterators}, and the value of this + * field is the index of the original iterator which is the source of the sole (uncombined) point. + * + * If the value of this field is less than 0, it means that some combines are done at the current point, and {@link + * #currentTimeAndDimsPointer} is one of {@link #combinedTimeAndDimsPointersByOriginalIteratorIndex}. + */ + private int soleCurrentPointSourceOriginalIteratorIndex; + + @Nullable + private RowPointer nextRowPointer; + + RowCombiningTimeAndDimsIterator( + List originalIterators, + AggregatorFactory[] metricAggs, + List metricNames + ) + { + int numCombinedIterators = originalIterators.size(); + mergingIterator = new MergingRowIterator(originalIterators); + + markedRowPointersOfOriginalIterators = new TimeAndDimsPointer[numCombinedIterators]; + Arrays.setAll( + markedRowPointersOfOriginalIterators, + originalIteratorIndex -> { + TransformableRowIterator originalIterator = mergingIterator.getOriginalIterator(originalIteratorIndex); + return originalIterator != null ? originalIterator.getMarkedPointer() : null; + } + ); + + combinedMetricSelectors = new AggregateCombiner[metricAggs.length]; + Arrays.setAll(combinedMetricSelectors, metricIndex -> metricAggs[metricIndex].makeAggregateCombiner()); + combinedMetricNames = metricNames; + + combinedTimeAndDimsPointersByOriginalIteratorIndex = new TimeAndDimsPointer[numCombinedIterators]; + Arrays.setAll( + combinedTimeAndDimsPointersByOriginalIteratorIndex, + originalIteratorIndex -> { + TimeAndDimsPointer markedRowPointer = markedRowPointersOfOriginalIterators[originalIteratorIndex]; + if (markedRowPointer != null) { + return new TimeAndDimsPointer( + markedRowPointer.timestampSelector, + markedRowPointer.dimensionSelectors, + markedRowPointer.getDimensionHandlers(), + combinedMetricSelectors, + combinedMetricNames + ); + } else { + return null; + } + } + ); + + minCurrentlyCombinedRowNumByOriginalIteratorIndex = new int[numCombinedIterators]; + Arrays.fill(minCurrentlyCombinedRowNumByOriginalIteratorIndex, MIN_CURRENTLY_COMBINED_ROW_NUM_UNSET_VALUE); + maxCurrentlyCombinedRowNumByOriginalIteratorIndex = new int[numCombinedIterators]; + + if (mergingIterator.moveToNext()) { + nextRowPointer = mergingIterator.getPointer(); + } + } + + /** + * Clear the info about which rows (in which original iterators and which row nums within them) were combined on + * the previous step. + */ + private void clearCombinedRowsInfo() + { + for (int originalIteratorIndex = indexesOfCurrentlyCombinedOriginalIterators.nextSetBit(0); + originalIteratorIndex >= 0; + originalIteratorIndex = indexesOfCurrentlyCombinedOriginalIterators.nextSetBit(originalIteratorIndex + 1)) { + minCurrentlyCombinedRowNumByOriginalIteratorIndex[originalIteratorIndex] = + MIN_CURRENTLY_COMBINED_ROW_NUM_UNSET_VALUE; + } + indexesOfCurrentlyCombinedOriginalIterators.clear(); + } + + /** + * Warning: this method and {@link #startNewTimeAndDims} have just ~25 lines of code, but their logic is very + * convoluted and hard to understand. It could be especially confusing to try to understand it via debug. + */ + @Override + public boolean moveToNext() + { + clearCombinedRowsInfo(); + if (nextRowPointer == null) { + currentTimeAndDimsPointer = null; + return false; + } + // This line implicitly uses the property of RowIterator.getPointer() (see [*] below), that it's still valid after + // RowPointer.moveToNext() returns false. mergingIterator.moveToNext() could have returned false during the previous + // call to this method, RowCombiningTimeAndDimsIterator.moveToNext(). + startNewTimeAndDims(nextRowPointer); + nextRowPointer = null; + // [1] -- see comment in startNewTimeAndDims() + mergingIterator.mark(); + // [2] -- see comment in startNewTimeAndDims() + while (mergingIterator.moveToNext()) { + if (mergingIterator.hasTimeAndDimsChangedSinceMark()) { + nextRowPointer = mergingIterator.getPointer(); // [*] + return true; + } else { + combineToCurrentTimeAndDims(mergingIterator.getPointer()); + } + } + // No more rows left in mergingIterator + nextRowPointer = null; + return true; + } + + /** + * This method doesn't assign one of {@link #combinedTimeAndDimsPointersByOriginalIteratorIndex} into {@link + * #currentTimeAndDimsPointer}, instead it uses one of {@link #markedRowPointersOfOriginalIterators}, see the javadoc + * of this field for explanation. + */ + private void startNewTimeAndDims(RowPointer rowPointer) + { + int originalIteratorIndex = rowPointer.getIndexNum(); + // Note: at the moment when this operation is performed, markedRowPointersOfOriginalIterators[originalIteratorIndex] + // doesn't yet contain actual current dimension and metric values. startNewTimeAndDims() is called from + // moveToNext(), see above. Later in the code of moveToNext(), mergingIterator.mark() [1] is called, and then + // mergingIterator.moveToNext() [2]. This will make MergingRowIterator.moveToNext() implementation (see it's code) + // to call mark() on the current head iteratator, and only after that + // markedRowPointersOfOriginalIterators[originalIteratorIndex] will have correct values. So by the time when + // moveToNext() (from where this method is called) exits, and before getPointer() could be called by the user of + // this class, it will have correct values. + currentTimeAndDimsPointer = markedRowPointersOfOriginalIterators[originalIteratorIndex]; + soleCurrentPointSourceOriginalIteratorIndex = originalIteratorIndex; + indexesOfCurrentlyCombinedOriginalIterators.set(originalIteratorIndex); + int rowNum = rowPointer.getRowNum(); + minCurrentlyCombinedRowNumByOriginalIteratorIndex[originalIteratorIndex] = rowNum; + maxCurrentlyCombinedRowNumByOriginalIteratorIndex[originalIteratorIndex] = rowNum; + } + + private void combineToCurrentTimeAndDims(RowPointer rowPointer) + { + int soleCurrentPointSourceOriginalIteratorIndex = this.soleCurrentPointSourceOriginalIteratorIndex; + if (soleCurrentPointSourceOriginalIteratorIndex >= 0) { + TimeAndDimsPointer currentRowPointer = this.currentTimeAndDimsPointer; + assert currentRowPointer != null; + resetCombinedMetrics(currentRowPointer); + currentTimeAndDimsPointer = + combinedTimeAndDimsPointersByOriginalIteratorIndex[soleCurrentPointSourceOriginalIteratorIndex]; + this.soleCurrentPointSourceOriginalIteratorIndex = -1; + } + + int originalIteratorIndex = rowPointer.getIndexNum(); + indexesOfCurrentlyCombinedOriginalIterators.set(originalIteratorIndex); + int rowNum = rowPointer.getRowNum(); + if (minCurrentlyCombinedRowNumByOriginalIteratorIndex[originalIteratorIndex] < 0) { + minCurrentlyCombinedRowNumByOriginalIteratorIndex[originalIteratorIndex] = rowNum; + } + maxCurrentlyCombinedRowNumByOriginalIteratorIndex[originalIteratorIndex] = rowNum; + + foldMetrics(rowPointer); + } + + private void resetCombinedMetrics(TimeAndDimsPointer currentRowPointer) + { + for (int metricIndex = 0; metricIndex < combinedMetricSelectors.length; metricIndex++) { + combinedMetricSelectors[metricIndex].reset(currentRowPointer.getMetricSelector(metricIndex)); + } + } + + private void foldMetrics(RowPointer rowPointer) + { + for (int metricIndex = 0; metricIndex < combinedMetricSelectors.length; metricIndex++) { + combinedMetricSelectors[metricIndex].fold(rowPointer.getMetricSelector(metricIndex)); + } + } + + @Override + public TimeAndDimsPointer getPointer() + { + return Objects.requireNonNull(currentTimeAndDimsPointer); + } + + /** + * Gets the next index of iterators (as provided in the List in constructor of RowCombiningTimeAndDimsIterator), + * that was the source of one or more points, that are combined to produce the current {@link #getPointer()} point. + * + * Should be used a-la {@link BitSet} iteration: + * for (int originalIteratorIndex = nextCurrentlyCombinedOriginalIteratorIndex(0); + * originalIteratorIndex >= 0; + * originalIteratorIndex = nextCurrentlyCombinedOriginalIteratorIndex(originalIteratorIndex + 1)) { + * ... + * } + */ + int nextCurrentlyCombinedOriginalIteratorIndex(int fromIndex) + { + return indexesOfCurrentlyCombinedOriginalIterators.nextSetBit(fromIndex); + } + + /** + * See Javadoc of {@link #minCurrentlyCombinedRowNumByOriginalIteratorIndex} for explanation. + */ + int getMinCurrentlyCombinedRowNumByOriginalIteratorIndex(int originalIteratorIndex) + { + return minCurrentlyCombinedRowNumByOriginalIteratorIndex[originalIteratorIndex]; + } + + /** + * See Javadoc of {@link #minCurrentlyCombinedRowNumByOriginalIteratorIndex} for explanation. + */ + int getMaxCurrentlyCombinedRowNumByOriginalIteratorIndex(int originalIteratorIndex) + { + return maxCurrentlyCombinedRowNumByOriginalIteratorIndex[originalIteratorIndex]; + } + + @Override + public void close() + { + mergingIterator.close(); + } +} diff --git a/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java b/processing/src/main/java/io/druid/segment/RowFilteringIndexAdapter.java similarity index 58% rename from processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java rename to processing/src/main/java/io/druid/segment/RowFilteringIndexAdapter.java index 590f48b3d926..e93753a1af4b 100644 --- a/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/RowFilteringIndexAdapter.java @@ -19,23 +19,22 @@ package io.druid.segment; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.data.BitmapValues; import io.druid.segment.data.Indexed; import org.joda.time.Interval; -import java.util.Map; +import java.util.List; +import java.util.function.Predicate; /** */ -public class RowboatFilteringIndexAdapter implements IndexableAdapter +public class RowFilteringIndexAdapter implements IndexableAdapter { - private final IndexableAdapter baseAdapter; - private final Predicate filter; + private final QueryableIndexIndexableAdapter baseAdapter; + private final Predicate filter; - public RowboatFilteringIndexAdapter(IndexableAdapter baseAdapter, Predicate filter) + public RowFilteringIndexAdapter(QueryableIndexIndexableAdapter baseAdapter, Predicate filter) { this.baseAdapter = baseAdapter; this.filter = filter; @@ -54,27 +53,51 @@ public int getNumRows() } @Override - public Indexed getDimensionNames() + public List getDimensionNames() { return baseAdapter.getDimensionNames(); } @Override - public Indexed getMetricNames() + public List getMetricNames() { return baseAdapter.getMetricNames(); } @Override - public Indexed getDimValueLookup(String dimension) + public > Indexed getDimValueLookup(String dimension) { return baseAdapter.getDimValueLookup(dimension); } @Override - public Iterable getRows() + public TransformableRowIterator getRows() { - return Iterables.filter(baseAdapter.getRows(), filter); + QueryableIndexIndexableAdapter.RowIteratorImpl baseRowIterator = baseAdapter.getRows(); + return new ForwardingRowIterator(baseRowIterator) + { + /** + * This memoization is needed to conform to {@link RowIterator#getPointer()} specification. + */ + private boolean memoizedOffset = false; + + @Override + public boolean moveToNext() + { + while (baseRowIterator.moveToNext()) { + if (filter.test(baseRowIterator.getPointer())) { + baseRowIterator.memoizeOffset(); + memoizedOffset = true; + return true; + } + } + // Setting back to the last valid offset in this iterator, as required by RowIterator.getPointer() spec. + if (memoizedOffset) { + baseRowIterator.resetToMemoizedOffset(); + } + return false; + } + }; } @Override @@ -100,10 +123,4 @@ public Metadata getMetadata() { return baseAdapter.getMetadata(); } - - @Override - public Map getDimensionHandlers() - { - return baseAdapter.getDimensionHandlers(); - } } diff --git a/processing/src/main/java/io/druid/segment/RowIterator.java b/processing/src/main/java/io/druid/segment/RowIterator.java new file mode 100644 index 000000000000..e3825906fc5f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/RowIterator.java @@ -0,0 +1,68 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +/** + * Extension of {@link TimeAndDimsIterator}, specialized for {@link RowPointer} instead of {@link TimeAndDimsPointer}. + * + * Also, RowIterator encapsulates tracking of data point changes between {@link #moveToNext()} calls via {@link #mark()} + * and {@link #hasTimeAndDimsChangedSinceMark()} methods. This functionality is used in {@link + * RowCombiningTimeAndDimsIterator}, and also internally in {@link MergingRowIterator} to reduce the number of + * comparisons that this class makes. This functionality is added directly to {@link RowIterator} interface rather than + * left to be implemented externally to this interface, because it's inefficient to do the latter with "generic" + * RowIterator, because of {@link TimeAndDimsPointer} allocation-free design, that reuses objects. On the other hand, + * some implementations of RowIterator allow to optimize {@link #mark()} and {@link #hasTimeAndDimsChangedSinceMark()}. + */ +public interface RowIterator extends TimeAndDimsIterator +{ + /** + * "Memoizes" the data point, to which {@link #getPointer()} currently points. If the last call to {@link + * #moveToNext()} returned {@code false}, the behaviour of this method is undefined, e. g. it may throw a runtime + * exception. + */ + void mark(); + + /** + * Compares the "memoized" data point from the last {@link #mark()} call with the data point, to which {@link + * #getPointer()} currently points. Comparison is made in terms of {@link TimeAndDimsPointer#compareTo} contract. + * + * If {@link #mark()} has never been called, or the last call to {@link #moveToNext()} returned {@code false}, the + * behaviour of this method is undefined: it may arbitrarily return true, or false, or throw a runtime exception. + */ + boolean hasTimeAndDimsChangedSinceMark(); + + /** + * Returns a pointer to the current row. This method may return the same (in terms of referencial identity), + * as well as different object on any calls, but the row itself, to which the returned object points, changes + * after each {@link #moveToNext()} call that returns {@link true}. + * + * This method must not be called before ever calling to {@link #moveToNext()}. If the very first call to {@link + * #moveToNext()} returned {@code false}, the behaviour of this method is undefined (it may return a "wrong" pointer, + * null, throw an exception, etc.). + * + * If {@link #moveToNext()} returned {@code true} one or more times, and then eventually returned {@code false}, + * calling {@code getPointer()} after that should return a pointer pointing to the last valid row, as if {@code + * getPointer()} was called before the last (unsuccessful) call to {@link #moveToNext()}. In other words, unsuccessful + * {@link #moveToNext()} call doesn't "corrupt" the pointer. This property is used in {@link + * RowCombiningTimeAndDimsIterator#moveToNext}. + */ + @Override + RowPointer getPointer(); +} diff --git a/processing/src/main/java/io/druid/segment/incremental/TimeAndDimsHolder.java b/processing/src/main/java/io/druid/segment/RowNumCounter.java similarity index 54% rename from processing/src/main/java/io/druid/segment/incremental/TimeAndDimsHolder.java rename to processing/src/main/java/io/druid/segment/RowNumCounter.java index 3e080e4a877b..bd49e03f3c82 100644 --- a/processing/src/main/java/io/druid/segment/incremental/TimeAndDimsHolder.java +++ b/processing/src/main/java/io/druid/segment/RowNumCounter.java @@ -17,32 +17,34 @@ * under the License. */ -package io.druid.segment.incremental; +package io.druid.segment; + +import java.util.function.IntSupplier; /** - * This interface is the core "pointer" interface that is used to create {@link io.druid.segment.ColumnValueSelector}s - * over incremental index. It's counterpart for historical segments is {@link io.druid.segment.data.Offset}. + * Could be used as a simple "row number supplier" for {@link RowPointer} implementations. */ -public class TimeAndDimsHolder +public final class RowNumCounter implements IntSupplier { - IncrementalIndex.TimeAndDims currEntry = null; + private int rowNum = 0; + + public RowNumCounter() + { + } - public IncrementalIndex.TimeAndDims get() + public RowNumCounter(int initialRowNum) { - return currEntry; + this.rowNum = initialRowNum; } - public void set(IncrementalIndex.TimeAndDims currEntry) + @Override + public int getAsInt() { - this.currEntry = currEntry; + return rowNum; } - /** - * This method doesn't have well-defined semantics ("value" of what?), should be removed in favor of chaining - * get().getRowIndex(). - */ - public int getValue() + public void increment() { - return currEntry.getRowIndex(); + rowNum++; } } diff --git a/processing/src/main/java/io/druid/segment/RowPointer.java b/processing/src/main/java/io/druid/segment/RowPointer.java new file mode 100644 index 000000000000..b9c4beb7c7ac --- /dev/null +++ b/processing/src/main/java/io/druid/segment/RowPointer.java @@ -0,0 +1,94 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import java.util.List; +import java.util.function.IntSupplier; + +/** + * Extension of {@link TimeAndDimsPointer}, that keeps "row number" of the current data point in some collection of data + * points, that actually makes this data point to be called "row", and the collection, thus, "collection of rows". + * However, "row number" doesn't affect the {@link TimeAndDimsPointer#compareTo} contract. RowPointers could be compared + * to TimeAndDimsPointers interchangeably. + */ +public final class RowPointer extends TimeAndDimsPointer +{ + final IntSupplier rowNumPointer; + + /** + * This field is not a part of "row" abstraction. It is used only between {@link MergingRowIterator} and + * {@link RowCombiningTimeAndDimsIterator} to transmit "index numbers" without adding more referencing data + * structures, i. e. to improve locality. Otherwise, this field is just not used. + */ + private int indexNum; + + public RowPointer( + ColumnValueSelector timestampSelector, + ColumnValueSelector[] dimensionSelectors, + List dimensionHandlers, + ColumnValueSelector[] metricSelectors, + List metricNames, + IntSupplier rowNumPointer + ) + { + super(timestampSelector, dimensionSelectors, dimensionHandlers, metricSelectors, metricNames); + this.rowNumPointer = rowNumPointer; + } + + public int getRowNum() + { + return rowNumPointer.getAsInt(); + } + + @Override + RowPointer withDimensionSelectors(ColumnValueSelector[] newDimensionSelectors) + { + return new RowPointer( + timestampSelector, + newDimensionSelectors, + getDimensionHandlers(), + metricSelectors, + getMetricNames(), + rowNumPointer + ); + } + + void setIndexNum(int indexNum) + { + this.indexNum = indexNum; + } + + int getIndexNum() + { + return indexNum; + } + + @Override + public String toString() + { + return "RowPointer{" + + "indexNum=" + indexNum + + ", rowNumber=" + rowNumPointer.getAsInt() + + ", timestamp=" + getTimestamp() + + ", dimensions=" + getDimensionNamesToValuesForDebug() + + ", metrics=" + getMetricNamesToValuesForDebug() + + '}'; + } +} diff --git a/processing/src/main/java/io/druid/segment/Rowboat.java b/processing/src/main/java/io/druid/segment/Rowboat.java deleted file mode 100644 index c047fd1c567b..000000000000 --- a/processing/src/main/java/io/druid/segment/Rowboat.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment; - -import com.google.common.primitives.Ints; -import com.google.common.primitives.Longs; -import io.druid.java.util.common.DateTimes; -import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; -import it.unimi.dsi.fastutil.ints.IntRBTreeSet; -import it.unimi.dsi.fastutil.ints.IntSortedSet; - -import java.util.Arrays; - -public class Rowboat implements Comparable -{ - private final long timestamp; - private final Object[] dims; - private final Object[] metrics; - private final int rowNum; - private final Int2ObjectOpenHashMap comprisedRows; - private final DimensionHandler[] handlers; - - public Rowboat( - long timestamp, - Object[] dims, - Object[] metrics, - int rowNum, - DimensionHandler[] handlers - ) - { - this.timestamp = timestamp; - this.dims = dims; - this.metrics = metrics; - this.rowNum = rowNum; - this.handlers = handlers; - - this.comprisedRows = new Int2ObjectOpenHashMap<>(1); - } - - public long getTimestamp() - { - return timestamp; - } - - public Object[] getDims() - { - return dims; - } - - public Object[] getMetrics() - { - return metrics; - } - - public void addRow(int indexNum, int rowNum) - { - IntSortedSet rowNums = comprisedRows.get(indexNum); - if (rowNums == null) { - rowNums = new IntRBTreeSet(); - comprisedRows.put(indexNum, rowNums); - } - rowNums.add(rowNum); - } - - public Int2ObjectOpenHashMap getComprisedRows() - { - return comprisedRows; - } - - public DimensionHandler[] getHandlers() - { - return handlers; - } - - public int getRowNum() - { - return rowNum; - } - - @Override - public int compareTo(Rowboat rhs) - { - int retVal = Longs.compare(timestamp, rhs.timestamp); - - if (retVal == 0) { - retVal = Ints.compare(dims.length, rhs.dims.length); - } - - int index = 0; - while (retVal == 0 && index < dims.length) { - Object lhsVals = dims[index]; - Object rhsVals = rhs.dims[index]; - - if (lhsVals == null) { - if (rhsVals == null) { - index++; - continue; - } - return -1; - } - - if (rhsVals == null) { - return 1; - } - - DimensionHandler handler = handlers[index]; - retVal = handler.compareSortedEncodedKeyComponents(lhsVals, rhsVals); - ++index; - } - - return retVal; - } - - @Override - public String toString() - { - return "Rowboat{" + - "timestamp=" + DateTimes.utc(timestamp) + - ", dims=" + Arrays.deepToString(dims) + - ", metrics=" + Arrays.toString(metrics) + - ", comprisedRows=" + comprisedRows + - '}'; - } -} diff --git a/processing/src/main/java/io/druid/segment/NoFilterOffset.java b/processing/src/main/java/io/druid/segment/SimpleAscendingOffset.java similarity index 79% rename from processing/src/main/java/io/druid/segment/NoFilterOffset.java rename to processing/src/main/java/io/druid/segment/SimpleAscendingOffset.java index 8fdf37d1cdca..58c465ff2cf0 100644 --- a/processing/src/main/java/io/druid/segment/NoFilterOffset.java +++ b/processing/src/main/java/io/druid/segment/SimpleAscendingOffset.java @@ -23,19 +23,22 @@ import io.druid.segment.data.Offset; import io.druid.segment.data.ReadableOffset; -public class NoFilterOffset extends Offset +public class SimpleAscendingOffset extends Offset { private final int rowCount; - private final boolean descending; private final int initialOffset; private int currentOffset; - NoFilterOffset(int initialOffset, int rowCount, boolean descending) + SimpleAscendingOffset(int rowCount) + { + this(0, rowCount); + } + + private SimpleAscendingOffset(int initialOffset, int rowCount) { this.initialOffset = initialOffset; this.currentOffset = initialOffset; this.rowCount = rowCount; - this.descending = descending; } @Override @@ -50,6 +53,11 @@ public boolean withinBounds() return currentOffset < rowCount; } + void setCurrentOffset(int currentOffset) + { + this.currentOffset = currentOffset; + } + @Override public void reset() { @@ -65,24 +73,24 @@ public ReadableOffset getBaseReadableOffset() @Override public Offset clone() { - return new NoFilterOffset(currentOffset, rowCount, descending); + return new SimpleAscendingOffset(currentOffset, rowCount); } @Override public int getOffset() { - return descending ? rowCount - currentOffset - 1 : currentOffset; + return currentOffset; } @Override public String toString() { - return currentOffset + "/" + rowCount + (descending ? "(DSC)" : ""); + return currentOffset + "/" + rowCount; } @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - inspector.visit("descending", descending); + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/segment/SimpleDescendingOffset.java b/processing/src/main/java/io/druid/segment/SimpleDescendingOffset.java new file mode 100644 index 000000000000..2d66b9625628 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/SimpleDescendingOffset.java @@ -0,0 +1,91 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.data.Offset; +import io.druid.segment.data.ReadableOffset; + +public class SimpleDescendingOffset extends Offset +{ + private final int rowCount; + private final int initialOffset; + private int currentOffset; + + SimpleDescendingOffset(int rowCount) + { + this(rowCount - 1, rowCount); + } + + private SimpleDescendingOffset(int initialOffset, int rowCount) + { + this.rowCount = rowCount; + this.initialOffset = initialOffset; + this.currentOffset = initialOffset; + } + + @Override + public void increment() + { + currentOffset--; + } + + @Override + public boolean withinBounds() + { + return currentOffset >= 0; + } + + @Override + public void reset() + { + currentOffset = initialOffset; + } + + @Override + public ReadableOffset getBaseReadableOffset() + { + return this; + } + + @Override + public Offset clone() + { + return new SimpleDescendingOffset(currentOffset, rowCount); + } + + @Override + public int getOffset() + { + return currentOffset; + } + + @Override + public String toString() + { + return currentOffset + "/" + rowCount + "(DSC)"; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } +} diff --git a/processing/src/main/java/io/druid/segment/StorageAdapter.java b/processing/src/main/java/io/druid/segment/StorageAdapter.java index f6658082b490..f49d31cc39b4 100644 --- a/processing/src/main/java/io/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/StorageAdapter.java @@ -72,6 +72,7 @@ public interface StorageAdapter extends CursorFactory * @param column column name * @return type name */ + @Nullable String getColumnTypeName(String column); int getNumRows(); DateTime getMaxIngestedEventTime(); diff --git a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java index 8a2d79d5ecf2..a4c4a06e8a4f 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java @@ -19,23 +19,96 @@ package io.druid.segment; -import com.google.common.primitives.Ints; import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; import io.druid.java.util.common.ISE; -import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.DictionaryEncodedColumn; -import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.ZeroIndexedInts; +import io.druid.segment.selector.settable.SettableColumnValueSelector; +import io.druid.segment.selector.settable.SettableDimensionValueSelector; import io.druid.segment.writeout.SegmentWriteOutMedium; -import javax.annotation.Nullable; -import java.io.Closeable; -import java.lang.reflect.Array; -import java.util.Arrays; +import java.util.Comparator; public class StringDimensionHandler implements DimensionHandler { + + /** + * Compares {@link IndexedInts} lexicographically, with the exception that if a row contains only zeros (that's the + * index of null) at all positions, it is considered "null" as a whole and is "less" than any "non-null" row. Empty + * row (size is zero) is also considered "null". + * + * The implementation is a bit complicated because it tries to check each position of both rows only once. + */ + private static final Comparator DIMENSION_SELECTOR_COMPARATOR = (s1, s2) -> { + IndexedInts row1 = getRow(s1); + IndexedInts row2 = getRow(s2); + int len1 = row1.size(); + int len2 = row2.size(); + boolean row1IsNull = true; + boolean row2IsNull = true; + for (int i = 0; i < Math.min(len1, len2); i++) { + int v1 = row1.get(i); + row1IsNull &= v1 == 0; + int v2 = row2.get(i); + row2IsNull &= v2 == 0; + int valueDiff = Integer.compare(v1, v2); + if (valueDiff != 0) { + return valueDiff; + } + } + //noinspection SubtractionInCompareTo -- substraction is safe here, because lenghts or rows are small numbers. + int lenDiff = len1 - len2; + if (lenDiff == 0) { + return 0; + } else { + if (!row1IsNull || !row2IsNull) { + return lenDiff; + } else { + return compareRestNulls(row1, len1, row2, len2); + } + } + }; + + private static int compareRestNulls(IndexedInts row1, int len1, IndexedInts row2, int len2) + { + if (len1 < len2) { + for (int i = len1; i < len2; i++) { + if (row2.get(i) != 0) { + return -1; + } + } + } else { + for (int i = len2; i < len1; i++) { + if (row1.get(i) != 0) { + return 1; + } + } + } + return 0; + } + + /** + * Value for absent column, i. e. {@link NilColumnValueSelector}, should be equivalent to [null] during index merging. + * + * During index merging, if one of the merged indexes has absent columns, {@link StringDimensionMergerV9} ensures + * that null value is present, and it has index = 0 after sorting, because sorting puts null first. See {@link + * StringDimensionMergerV9#hasNull} and the place where it is assigned. + */ + private static IndexedInts getRow(ColumnValueSelector s) + { + if (s instanceof DimensionSelector) { + return ((DimensionSelector) s).getRow(); + } else if (s instanceof NilColumnValueSelector) { + return ZeroIndexedInts.instance(); + } else { + throw new ISE( + "ColumnValueSelector[%s], only DimensionSelector or NilColumnValueSelector is supported", + s.getClass() + ); + } + } + private final String dimensionName; private final MultiValueHandling multiValueHandling; private final boolean hasBitmapIndexes; @@ -66,145 +139,15 @@ public int getLengthOfEncodedKeyComponent(int[] dimVals) } @Override - public int compareSortedEncodedKeyComponents(int[] lhs, int[] rhs) - { - int lhsLen = lhs.length; - int rhsLen = rhs.length; - - int retVal = Ints.compare(lhsLen, rhsLen); - - int valsIndex = 0; - while (retVal == 0 && valsIndex < lhsLen) { - retVal = Ints.compare(lhs[valsIndex], rhs[valsIndex]); - ++valsIndex; - } - return retVal; - } - - private boolean isNullRow(@Nullable int[] row, Indexed encodings) - { - if (row == null) { - return true; - } - for (int value : row) { - if (encodings.get(value) != null) { - // Non-Null value - return false; - } - } - return true; - } - - @Override - public void validateSortedEncodedKeyComponents( - int[] lhs, - int[] rhs, - Indexed lhsEncodings, - Indexed rhsEncodings - ) throws SegmentValidationException - { - if (lhs == null || rhs == null) { - if (!isNullRow(lhs, lhsEncodings) || !isNullRow(rhs, rhsEncodings)) { - throw new SegmentValidationException( - "Expected nulls, found %s and %s", - Arrays.toString(lhs), - Arrays.toString(rhs) - ); - } else { - return; - } - } - - int lhsLen = Array.getLength(lhs); - int rhsLen = Array.getLength(rhs); - - if (lhsLen != rhsLen) { - // Might be OK if one of them has null. This occurs in IndexMakerTest - if (lhsLen == 0 && rhsLen == 1) { - final String dimValName = rhsEncodings.get(rhs[0]); - if (dimValName == null) { - return; - } else { - throw new SegmentValidationException( - "Dim [%s] value [%s] is not null", - dimensionName, - dimValName - ); - } - } else if (rhsLen == 0 && lhsLen == 1) { - final String dimValName = lhsEncodings.get(lhs[0]); - if (dimValName == null) { - return; - } else { - throw new SegmentValidationException( - "Dim [%s] value [%s] is not null", - dimensionName, - dimValName - ); - } - } else { - throw new SegmentValidationException( - "Dim [%s] value lengths not equal. Expected %d found %d", - dimensionName, - lhsLen, - rhsLen - ); - } - } - - for (int j = 0; j < Math.max(lhsLen, rhsLen); ++j) { - final int dIdex1 = lhsLen <= j ? -1 : lhs[j]; - final int dIdex2 = rhsLen <= j ? -1 : rhs[j]; - - final String dim1ValName = dIdex1 < 0 ? null : lhsEncodings.get(dIdex1); - final String dim2ValName = dIdex2 < 0 ? null : rhsEncodings.get(dIdex2); - if ((dim1ValName == null) || (dim2ValName == null)) { - if ((dim1ValName == null) && (dim2ValName == null)) { - continue; - } else { - throw new SegmentValidationException( - "Dim [%s] value not equal. Expected [%s] found [%s]", - dimensionName, - dim1ValName, - dim2ValName - ); - } - } - - if (!dim1ValName.equals(dim2ValName)) { - throw new SegmentValidationException( - "Dim [%s] value not equal. Expected [%s] found [%s]", - dimensionName, - dim1ValName, - dim2ValName - ); - } - } - } - - @Override - public Closeable getSubColumn(Column column) + public Comparator getEncodedValueSelectorComparator() { - return column.getDictionaryEncoding(); + return DIMENSION_SELECTOR_COMPARATOR; } @Override - public int[] getEncodedKeyComponentFromColumn(Closeable column, int currRow) + public SettableColumnValueSelector makeNewSettableEncodedValueSelector() { - DictionaryEncodedColumn dict = (DictionaryEncodedColumn) column; - int[] theVals; - if (dict.hasMultipleValues()) { - final IndexedInts dimVals = dict.getMultiValueRow(currRow); - theVals = new int[dimVals.size()]; - for (int i = 0; i < theVals.length; ++i) { - theVals[i] = dimVals.get(i); - } - } else { - theVals = new int[1]; - theVals[0] = dict.getSingleValueRow(currRow); - } - - return theVals; + return new SettableDimensionValueSelector(); } @Override @@ -214,7 +157,7 @@ public DimensionIndexer makeIndexer() } @Override - public DimensionMergerV9 makeMerger( + public DimensionMergerV9 makeMerger( IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index 68457c5f4982..ee8920102129 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -40,7 +40,8 @@ import io.druid.segment.data.IndexedIterable; import io.druid.segment.filter.BooleanValueMatcher; import io.druid.segment.incremental.IncrementalIndex; -import io.druid.segment.incremental.TimeAndDimsHolder; +import io.druid.segment.incremental.IncrementalIndexRow; +import io.druid.segment.incremental.IncrementalIndexRowHolder; import it.unimi.dsi.fastutil.ints.IntArrays; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.objects.Object2IntMap; @@ -69,7 +70,9 @@ private static String emptytoNullIfNeeded(Object o) private static class DimensionDictionary { + @Nullable private String minValue = null; + @Nullable private String maxValue = null; private int idForNull = ABSENT_VALUE_ID; @@ -112,7 +115,7 @@ public int size() } } - public int add(String originalValue) + public int add(@Nullable String originalValue) { synchronized (lock) { if (originalValue == null) { @@ -202,6 +205,7 @@ public String getValueFromSortedId(int index) private final MultiValueHandling multiValueHandling; private final boolean hasBitmapIndexes; private SortedDimensionDictionary sortedLookup; + private boolean hasMultipleValues = false; public StringDimensionIndexer(MultiValueHandling multiValueHandling, boolean hasBitmapIndexes) { @@ -227,6 +231,7 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boole } else if (dimValuesList.size() == 1) { encodedDimensionValues = new int[]{dimLookup.add(emptytoNullIfNeeded(dimValuesList.get(0)))}; } else { + hasMultipleValues = true; final String[] dimensionValues = new String[dimValuesList.size()]; for (int i = 0; i < dimValuesList.size(); i++) { dimensionValues[i] = emptytoNullIfNeeded(dimValuesList.get(i)); @@ -378,7 +383,7 @@ public int getUnsortedEncodedKeyComponentHashCode(int[] key) @Override public DimensionSelector makeDimensionSelector( final DimensionSpec spec, - final TimeAndDimsHolder currEntry, + final IncrementalIndexRowHolder currEntry, final IncrementalIndex.DimensionDesc desc ) { @@ -409,18 +414,23 @@ public IndexedInts getRow() // usually due to currEntry's rowIndex is smaller than the row's rowIndex in which this dim first appears if (indices == null || indices.length == 0) { - final int nullId = getEncodedValue(null, false); - if (nullId > -1) { - if (nullIdIntArray == null) { - nullIdIntArray = new int[] {nullId}; - } - row = nullIdIntArray; - rowSize = 1; - } else { - // doesn't contain nullId, then empty array is used - // Choose to use ArrayBasedIndexedInts later, instead of special "empty" IndexedInts, for monomorphism + if (hasMultipleValues) { row = IntArrays.EMPTY_ARRAY; rowSize = 0; + } else { + final int nullId = getEncodedValue(null, false); + if (nullId > -1) { + if (nullIdIntArray == null) { + nullIdIntArray = new int[]{nullId}; + } + row = nullIdIntArray; + rowSize = 1; + } else { + // doesn't contain nullId, then empty array is used + // Choose to use ArrayBasedIndexedInts later, instead of special "empty" IndexedInts, for monomorphism + row = IntArrays.EMPTY_ARRAY; + rowSize = 0; + } } } @@ -558,7 +568,7 @@ public int lookupId(String name) @Override public Object getObject() { - IncrementalIndex.TimeAndDims key = currEntry.get(); + IncrementalIndexRow key = currEntry.get(); if (key == null) { return null; } @@ -592,7 +602,7 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public ColumnValueSelector makeColumnValueSelector( - TimeAndDimsHolder currEntry, + IncrementalIndexRowHolder currEntry, IncrementalIndex.DimensionDesc desc ) { @@ -627,14 +637,87 @@ public Object convertUnsortedEncodedKeyComponentToActualArrayOrList(int[] key, b } @Override - public int[] convertUnsortedEncodedKeyComponentToSortedEncodedKeyComponent(int[] key) + public ColumnValueSelector convertUnsortedValuesToSorted(ColumnValueSelector selectorWithUnsortedValues) { - int[] sortedDimVals = new int[key.length]; - for (int i = 0; i < key.length; ++i) { - // The encoded values in the TimeAndDims key are not sorted based on their final unencoded values, so need this lookup. - sortedDimVals[i] = getSortedEncodedValueFromUnsorted(key[i]); + DimensionSelector dimSelectorWithUnsortedValues = (DimensionSelector) selectorWithUnsortedValues; + class SortedDimensionSelector implements DimensionSelector, IndexedInts + { + @Override + public int size() + { + return dimSelectorWithUnsortedValues.getRow().size(); + } + + @Override + public int get(int index) + { + return sortedLookup().getSortedIdFromUnsortedId(dimSelectorWithUnsortedValues.getRow().get(index)); + } + + @Override + public IndexedInts getRow() + { + return this; + } + + @Override + public ValueMatcher makeValueMatcher(@Nullable String value) + { + throw new UnsupportedOperationException(); + } + + @Override + public ValueMatcher makeValueMatcher(Predicate predicate) + { + throw new UnsupportedOperationException(); + } + + @Override + public int getValueCardinality() + { + return dimSelectorWithUnsortedValues.getValueCardinality(); + } + + @Nullable + @Override + public String lookupName(int id) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean nameLookupPossibleInAdvance() + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public IdLookup idLookup() + { + throw new UnsupportedOperationException(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("dimSelectorWithUnsortedValues", dimSelectorWithUnsortedValues); + } + + @Nullable + @Override + public Object getObject() + { + return dimSelectorWithUnsortedValues.getObject(); + } + + @Override + public Class classOfObject() + { + return dimSelectorWithUnsortedValues.classOfObject(); + } } - return sortedDimVals; + return new SortedDimensionSelector(); } @Override diff --git a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java index ae93772128de..f7cb2b141f57 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java @@ -19,6 +19,7 @@ package io.druid.segment; +import com.google.common.base.Predicate; import com.google.common.base.Splitter; import com.google.common.collect.Lists; import io.druid.collections.bitmap.BitmapFactory; @@ -31,6 +32,8 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; +import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; @@ -39,12 +42,15 @@ import io.druid.segment.data.BitmapValues; import io.druid.segment.data.ByteBufferWriter; import io.druid.segment.data.ColumnarIntsSerializer; +import io.druid.segment.data.ColumnarMultiIntsSerializer; import io.druid.segment.data.CompressedVSizeColumnarIntsSerializer; import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.ImmutableRTreeObjectStrategy; import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.SingleValueColumnarIntsSerializer; import io.druid.segment.data.V3CompressedVSizeColumnarMultiIntsSerializer; import io.druid.segment.data.VSizeColumnarIntsSerializer; import io.druid.segment.data.VSizeColumnarMultiIntsSerializer; @@ -54,23 +60,20 @@ import it.unimi.dsi.fastutil.ints.IntIterator; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.IOException; import java.nio.IntBuffer; import java.util.ArrayList; import java.util.List; -public class StringDimensionMergerV9 implements DimensionMergerV9 +public class StringDimensionMergerV9 implements DimensionMergerV9 { private static final Logger log = new Logger(StringDimensionMergerV9.class); - protected static final Indexed NULL_STR_DIM_VAL = new ArrayIndexed<>( - new String[]{(String) null}, - String.class - ); - protected static final int[] NULL_STR_DIM_ARRAY = new int[]{0}; + private static final Indexed NULL_STR_DIM_VAL = new ArrayIndexed<>(new String[]{null}, String.class); private static final Splitter SPLITTER = Splitter.on(","); - private ColumnarIntsSerializer encodedValueWriter; + private ColumnarIntsSerializer encodedValueSerializer; private String dimensionName; private GenericIndexedWriter dictionaryWriter; @@ -80,7 +83,6 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 private ByteBufferWriter spatialWriter; private ArrayList dimConversions; private int cardinality = 0; - private boolean convertMissingValues = false; private boolean hasNull = false; private MutableBitmap nullRowsBitmap; private final SegmentWriteOutMedium segmentWriteOutMedium; @@ -108,7 +110,7 @@ public StringDimensionMergerV9( } @Override - public void writeMergedValueMetadata(List adapters) throws IOException + public void writeMergedValueDictionary(List adapters) throws IOException { boolean dimHasValues = false; boolean dimAbsentFromSomeIndex = false; @@ -126,8 +128,8 @@ public void writeMergedValueMetadata(List adapters) throws IOE Indexed dimValueLookup = null; Indexed[] dimValueLookups = new Indexed[adapters.size() + 1]; for (int i = 0; i < adapters.size(); i++) { - Indexed dimValues = (Indexed) adapters.get(i).getDimValueLookup(dimensionName); - if (!isNullColumn(dimValues)) { + Indexed dimValues = adapters.get(i).getDimValueLookup(dimensionName); + if (dimValues != null && !allNull(dimValues)) { dimHasValues = true; hasNull |= dimValues.indexOf(null) >= 0; dimValueLookups[i] = dimValueLookup = dimValues; @@ -137,12 +139,12 @@ public void writeMergedValueMetadata(List adapters) throws IOE } } - convertMissingValues = dimHasValues && dimAbsentFromSomeIndex; + boolean convertMissingValues = dimHasValues && dimAbsentFromSomeIndex; /* * Ensure the empty str is always in the dictionary if the dimension was missing from one index but * has non-null values in another index. - * This is done so that MMappedIndexRowIterable can convert null columns to empty strings + * This is done so that IndexMerger.toMergedIndexRowIterator() can convert null columns to empty strings * later on, to allow rows from indexes without a particular dimension to merge correctly with * rows from indexes with null/empty str values for that dimension. */ @@ -202,78 +204,174 @@ protected void setupEncodedValueWriter() throws IOException String filenameBase = StringUtils.format("%s.forward_dim", dimensionName); if (capabilities.hasMultipleValues()) { if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { - encodedValueWriter = V3CompressedVSizeColumnarMultiIntsSerializer.create( + encodedValueSerializer = V3CompressedVSizeColumnarMultiIntsSerializer.create( segmentWriteOutMedium, filenameBase, cardinality, compressionStrategy ); } else { - encodedValueWriter = new VSizeColumnarMultiIntsSerializer(segmentWriteOutMedium, cardinality); + encodedValueSerializer = new VSizeColumnarMultiIntsSerializer(segmentWriteOutMedium, cardinality); } } else { if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { - encodedValueWriter = CompressedVSizeColumnarIntsSerializer.create( + encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create( segmentWriteOutMedium, filenameBase, cardinality, compressionStrategy ); } else { - encodedValueWriter = new VSizeColumnarIntsSerializer(segmentWriteOutMedium, cardinality); + encodedValueSerializer = new VSizeColumnarIntsSerializer(segmentWriteOutMedium, cardinality); } } - encodedValueWriter.open(); + encodedValueSerializer.open(); } - @Override - public int[] convertSegmentRowValuesToMergedRowValues(int[] segmentRow, int segmentIndexNumber) + public ColumnValueSelector convertSortedSegmentRowValuesToMergedRowValues(int segmentIndex, ColumnValueSelector source) { - int[] dimVals = segmentRow; - // For strings, convert missing values to null/empty if conversion flag is set - // But if bitmap/dictionary is not used, always convert missing to 0 - if (dimVals == null) { - return convertMissingValues ? NULL_STR_DIM_ARRAY : null; + IntBuffer converter = dimConversions.get(segmentIndex); + if (converter == null) { + return source; } + DimensionSelector sourceDimensionSelector = (DimensionSelector) source; - int[] newDimVals = new int[dimVals.length]; - IntBuffer converter = dimConversions.get(segmentIndexNumber); + IndexedInts convertedRow = new IndexedInts() + { + @Override + public int size() + { + return sourceDimensionSelector.getRow().size(); + } - for (int i = 0; i < dimVals.length; i++) { - if (converter != null) { - newDimVals[i] = converter.get(dimVals[i]); - } else { - newDimVals[i] = dimVals[i]; + @Override + public int get(int index) + { + return converter.get(sourceDimensionSelector.getRow().get(index)); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("source", source); + inspector.visit("converter", converter); + } + }; + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + return convertedRow; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("convertedRow", convertedRow); + } + + @Override + public ValueMatcher makeValueMatcher(String value) + { + throw new UnsupportedOperationException(); + } + + @Override + public ValueMatcher makeValueMatcher(Predicate predicate) + { + throw new UnsupportedOperationException(); + } + + @Override + public int getValueCardinality() + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public String lookupName(int id) + { + throw new UnsupportedOperationException(); } - } - return newDimVals; + @Override + public boolean nameLookupPossibleInAdvance() + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public IdLookup idLookup() + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public Object getObject() + { + return sourceDimensionSelector.getObject(); + } + + @Override + public Class classOfObject() + { + return sourceDimensionSelector.classOfObject(); + } + }; } @Override - public void processMergedRow(int[] rowValues) throws IOException + public void processMergedRow(ColumnValueSelector selector) throws IOException { - int[] vals = rowValues; - if (vals == null || vals.length == 0) { + IndexedInts row = getRow(selector); + int rowSize = row.size(); + if (rowSize == 0) { nullRowsBitmap.add(rowCount); - } else if (hasNull && vals.length == 1 && (vals[0]) == 0) { - // Dictionary encoded, so it's safe to cast dim value to integer - // If this dimension has the null/empty str in its dictionary, a row with a single-valued dimension - // that matches the null/empty str's dictionary ID should also be added to nullRowBitmap. + } else if (hasNull && isNullRow(row, rowSize)) { + // If this dimension has the null/empty str in its dictionary, a row with nulls at all positions should also be + // added to nullRowBitmap. nullRowsBitmap.add(rowCount); } - processMergedRowHelper(vals); + if (encodedValueSerializer instanceof ColumnarMultiIntsSerializer) { + ((ColumnarMultiIntsSerializer) encodedValueSerializer).addValues(row); + } else { + int value = row.size() == 0 ? 0 : row.get(0); + ((SingleValueColumnarIntsSerializer) encodedValueSerializer).addValue(value); + } rowCount++; } - protected void processMergedRowHelper(int[] vals) throws IOException + private static IndexedInts getRow(ColumnValueSelector s) + { + if (s instanceof DimensionSelector) { + return ((DimensionSelector) s).getRow(); + } else if (s instanceof NilColumnValueSelector) { + return IndexedInts.empty(); + } else { + throw new ISE( + "ColumnValueSelector[%s], only DimensionSelector or NilColumnValueSelector is supported", + s.getClass() + ); + } + } + + private static boolean isNullRow(IndexedInts row, int size) { - encodedValueWriter.add(vals); + for (int i = 0; i < size; i++) { + if (row.get(i) != 0) { + return false; + } + } + return true; } @Override - public void writeIndexes(List segmentRowNumConversions) throws IOException + public void writeIndexes(@Nullable List segmentRowNumConversions) throws IOException { if (!capabilities.hasBitmapIndexes()) { return; @@ -336,7 +434,7 @@ public void writeIndexes(List segmentRowNumConversions) throws IOExce } void mergeBitmaps( - List segmentRowNumConversions, + @Nullable List segmentRowNumConversions, BitmapFactory bmpFactory, RTree tree, boolean hasSpatial, @@ -344,22 +442,27 @@ void mergeBitmaps( int dictId ) throws IOException { - List convertedInvertedIndexesToMerge = Lists.newArrayListWithCapacity(adapters.size()); + List convertedInvertedIndexesToMerge = Lists.newArrayListWithCapacity(adapters.size()); for (int j = 0; j < adapters.size(); ++j) { int seekedDictId = dictIdSeeker[j].seek(dictId); if (seekedDictId != IndexSeeker.NOT_EXIST) { - convertedInvertedIndexesToMerge.add( - new ConvertingBitmapValues( - adapters.get(j).getBitmapValues(dimensionName, seekedDictId), - segmentRowNumConversions.get(j) - ) - ); + IntIterable values; + if (segmentRowNumConversions != null) { + values = new ConvertingBitmapValues( + adapters.get(j).getBitmapValues(dimensionName, seekedDictId), + segmentRowNumConversions.get(j) + ); + } else { + BitmapValues bitmapValues = adapters.get(j).getBitmapValues(dimensionName, seekedDictId); + values = bitmapValues::iterator; + } + convertedInvertedIndexesToMerge.add(values); } } MutableBitmap mergedIndexes = bmpFactory.makeEmptyMutableBitmap(); List convertedInvertedIndexesIterators = new ArrayList<>(convertedInvertedIndexesToMerge.size()); - for (ConvertingBitmapValues convertedInvertedIndexes : convertedInvertedIndexesToMerge) { + for (IntIterable convertedInvertedIndexes : convertedInvertedIndexesToMerge) { convertedInvertedIndexesIterators.add(convertedInvertedIndexes.iterator()); } @@ -416,7 +519,7 @@ public ColumnDescriptor makeColumnDescriptor() .serializerBuilder() .withDictionary(dictionaryWriter) .withValue( - encodedValueWriter, + encodedValueSerializer, hasMultiValue, compressionStrategy != CompressionStrategy.UNCOMPRESSED ) @@ -564,20 +667,17 @@ protected IndexSeeker[] toIndexSeekers( if (dimConversion != null) { seekers[i] = new IndexSeekerWithConversion((IntBuffer) dimConversion.asReadOnlyBuffer().rewind()); } else { - Indexed dimValueLookup = (Indexed) adapters.get(i).getDimValueLookup(dimension); + Indexed dimValueLookup = adapters.get(i).getDimValueLookup(dimension); seekers[i] = new IndexSeekerWithoutConversion(dimValueLookup == null ? 0 : dimValueLookup.size()); } } return seekers; } - protected boolean isNullColumn(Iterable dimValues) + private boolean allNull(Indexed dimValues) { - if (dimValues == null) { - return true; - } - for (String val : dimValues) { - if (val != null) { + for (int i = 0, size = dimValues.size(); i < size; i++) { + if (dimValues.get(i) != null) { return false; } } diff --git a/processing/src/main/java/io/druid/segment/TimeAndDimsIterator.java b/processing/src/main/java/io/druid/segment/TimeAndDimsIterator.java new file mode 100644 index 000000000000..4a0936639822 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/TimeAndDimsIterator.java @@ -0,0 +1,63 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import java.io.Closeable; + +/** + * TimeAndDimsIterator (in conjunction with {@link TimeAndDimsPointer}) is an {@link java.util.Iterator}-like + * abstraction, designed for allocation-free transformation, merging, combining and iteration over a stream of data + * points. + * + * Usage pattern: + * try (TimeAndDimsIterator iterator = obtainNewTimeAndDimsIteratorFromSomewhere()) { + * while (iterator.moveToNext()) { + * TimeAndDimsPointer pointer = iterator.getPointer(); + * doSomethingWithPointer(pointer); + * } + * } + */ +public interface TimeAndDimsIterator extends Closeable +{ + /** + * Moves iterator to the next data point. This method must be called before the first use of {@link #getPointer()}. + * As long as this method returns {@code true}, {@link #getPointer()} could be safely called; after this method + * returned {@code false}, this iterator is done, {@link #getPointer()} must _not_ be called, and {@link #close()} + * should be called. + */ + boolean moveToNext(); + + /** + * Returns a pointer to the current data point. This method may return the same (in terms of referencial identity), + * as well as different object on any calls, but the data point itself, to which the returned object points, changes + * after each {@link #moveToNext()} call that returns {@link true}. + * + * This method must not be called before ever calling to {@link #moveToNext()}. After a call to {@link #moveToNext()} + * returned {@code false}, the behaviour of this method is undefined (it may return a "wrong" pointer, null, + * throw an exception, etc.) + */ + TimeAndDimsPointer getPointer(); + + /** + * Closes any resources, associated with this iterator. + */ + @Override + void close(); +} diff --git a/processing/src/main/java/io/druid/segment/TimeAndDimsPointer.java b/processing/src/main/java/io/druid/segment/TimeAndDimsPointer.java new file mode 100644 index 000000000000..bf0962cc7f1d --- /dev/null +++ b/processing/src/main/java/io/druid/segment/TimeAndDimsPointer.java @@ -0,0 +1,217 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import com.google.common.base.Preconditions; +import io.druid.java.util.common.DateTimes; + +import javax.annotation.Nonnull; +import java.util.Arrays; +import java.util.Comparator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +/** + * TimeAndDimsPointer is used in conjunction with {@link TimeAndDimsIterator}, it's an _immutable_ object that points to + * different logical data points, as {@link TimeAndDimsIterator#moveToNext()} is called. + * + * TimeAndDimsPointers are comparable by time and dimension column values, but excluding metric column values, to + * facilicate ordering and rollup during merging of collections of rows (see {@link IndexMergerV9#merge} methods). + * + * The difference between "time and dims" and "row" abstraction (see {@link + * io.druid.segment.incremental.IncrementalIndexRow}, {@link RowPointer}) is that "time and dims" is logical composite + * of only time point and dimension and metric values, not tied to a specific position in any data structure (aka "row + * index"). + * + * TimeAndDimsPointer is conceptually similar to {@link Cursor}, but the latter is used for query execution rather than + * historical segments creation (as TimeAndDimsPointer). If those abstractions could be collapsed (and if it is + * worthwhile) is yet to be determined. + */ +public class TimeAndDimsPointer implements Comparable +{ + final ColumnValueSelector timestampSelector; + /** + * This collection of dimension selectors is stored as array rather than List in order to minimize indirection in hot + * spots, in particular in {@link #compareTo}. + * + * The same reasoning is applied to {@link #dimensionSelectorComparators} and {@link #metricSelectors}. + */ + final ColumnValueSelector[] dimensionSelectors; + private final List dimensionHandlers; + /** + * Because of polymorphic nature of {@link ColumnValueSelector}, a priori there are many ways to compare two arbitrary + * dimension column value selectors. dimensionSelectorComparators encapsulate the information how specifically we + * should compare ColumnValueSelectors in each dimension. See {@link + * DimensionHandler#getEncodedValueSelectorComparator()}. + */ + private final Comparator[] dimensionSelectorComparators; + final ColumnValueSelector[] metricSelectors; + private final List metricNames; + + /** + * TimeAndDimsPointer constructor intentionally takes dimensionSelectors and metricSelectors as arrays and doesn't + * copy them "defensively", to allow to reuse arrays during transformations of TimeAndDimsPointers and {@link + * RowPointer}s in some cases, particularly in {@link + * RowCombiningTimeAndDimsIterator#RowCombiningTimeAndDimsIterator}, in order to reduce the number of array objects + * tapped on each iteration during index merge process. + */ + TimeAndDimsPointer( + ColumnValueSelector timestampSelector, + ColumnValueSelector[] dimensionSelectors, + List dimensionHandlers, + ColumnValueSelector[] metricSelectors, + List metricNames + ) + { + this.timestampSelector = timestampSelector; + Preconditions.checkArgument(dimensionSelectors.length == dimensionHandlers.size()); + this.dimensionSelectors = dimensionSelectors; + this.dimensionHandlers = dimensionHandlers; + //noinspection unchecked + this.dimensionSelectorComparators = dimensionHandlers + .stream() + .map(DimensionHandler::getEncodedValueSelectorComparator) + .toArray(Comparator[]::new); + Preconditions.checkArgument(metricSelectors.length == metricNames.size()); + this.metricSelectors = metricSelectors; + this.metricNames = metricNames; + } + + public long getTimestamp() + { + return timestampSelector.getLong(); + } + + ColumnValueSelector getDimensionSelector(int dimIndex) + { + return dimensionSelectors[dimIndex]; + } + + int getNumDimensions() + { + return dimensionSelectors.length; + } + + List getDimensionHandlers() + { + return dimensionHandlers; + } + + ColumnValueSelector getMetricSelector(int metricIndex) + { + return metricSelectors[metricIndex]; + } + + public int getNumMetrics() + { + return metricSelectors.length; + } + + List getMetricNames() + { + return metricNames; + } + + TimeAndDimsPointer withDimensionSelectors(ColumnValueSelector[] newDimensionSelectors) + { + return new TimeAndDimsPointer( + timestampSelector, + newDimensionSelectors, + dimensionHandlers, + metricSelectors, + getMetricNames() + ); + } + + /** + * Compares time column value and dimension column values, but not metric column values. + */ + @Override + public int compareTo(@Nonnull TimeAndDimsPointer rhs) + { + long timestamp = getTimestamp(); + long rhsTimestamp = rhs.getTimestamp(); + int timestampDiff = Long.compare(timestamp, rhsTimestamp); + if (timestampDiff != 0) { + return timestampDiff; + } + for (int dimIndex = 0; dimIndex < dimensionSelectors.length; dimIndex++) { + int dimDiff = dimensionSelectorComparators[dimIndex].compare( + dimensionSelectors[dimIndex], + rhs.dimensionSelectors[dimIndex] + ); + if (dimDiff != 0) { + return dimDiff; + } + } + return 0; + } + + @SuppressWarnings("Contract") + @Override + public boolean equals(Object obj) + { + throw new UnsupportedOperationException("Should not compare TimeAndDimsPointers using equals(), only compareTo()"); + } + + @Override + public int hashCode() + { + throw new UnsupportedOperationException("Should not compute hashCode() on TimeAndDimsPointer"); + } + + Object[] getDimensionValuesForDebug() + { + return Arrays.stream(dimensionSelectors).map(ColumnValueSelector::getObject).toArray(); + } + + @Override + public String toString() + { + return "TimeAndDimsPointer{" + + "timestamp=" + DateTimes.utc(getTimestamp()) + + ", dimensions=" + getDimensionNamesToValuesForDebug() + + ", metrics=" + getMetricNamesToValuesForDebug() + + '}'; + } + + Map getDimensionNamesToValuesForDebug() + { + LinkedHashMap result = new LinkedHashMap<>(); + for (int i = 0; i < getNumDimensions(); i++) { + Object value = dimensionSelectors[i].getObject(); + if (value instanceof Object[]) { + value = Arrays.asList((Object[]) value); + } + result.put(dimensionHandlers.get(i).getDimensionName(), value); + } + return result; + } + + Map getMetricNamesToValuesForDebug() + { + LinkedHashMap result = new LinkedHashMap<>(); + for (int i = 0; i < getNumMetrics(); i++) { + result.put(metricNames.get(i), metricSelectors[i].getObject()); + } + return result; + } +} diff --git a/processing/src/main/java/io/druid/segment/TransformableRowIterator.java b/processing/src/main/java/io/druid/segment/TransformableRowIterator.java new file mode 100644 index 000000000000..a6b557d728fe --- /dev/null +++ b/processing/src/main/java/io/druid/segment/TransformableRowIterator.java @@ -0,0 +1,57 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +/** + * TransformableRowIterator tightens {@link RowIterator#getPointer()} contract, that allows to transform iterated + * rows without allocations on each iterations, and reuse the mechanics of the underlying iterator. See {@link + * IndexMerger#toMergedIndexRowIterator} for an example. + */ +public interface TransformableRowIterator extends RowIterator +{ + /** + * Returns a pointer to the current row. This method _always returns the same object_, but pointing to + * different row after each {@link #moveToNext()} call that returns {@link true}. + * + * Other aspects of the behaviour of this method are the same as in the generic {@link RowIterator#getPointer()} + * contract. + */ + @Override + RowPointer getPointer(); + + /** + * Returns a pointer to the row, that was the current row when {@link #mark()} was called for the last time. This + * method always returns the same object, but pointing to different rows after each {@link #mark()} call. + * + * This method must not be called before ever calling to {@link #moveToNext()}. If the very first call to {@link + * #moveToNext()} returned {@code false}, the behaviour of this method is undefined (it may return a "wrong" pointer, + * null, throw an exception, etc.). + * + * This method could be called before the first call to {@link #mark()} (it should return the same object as it is + * required to return after all subsequent {@link #mark()} calls), but the data of the returned pointer should not + * be accessed before the first call to {@link #mark()}. + * + * This method is used in {@link RowCombiningTimeAndDimsIterator} implementation. getMarkedPointer() returns {@link + * TimeAndDimsPointer} instead of {@link RowPointer} (like {@link #getPointer()}) merely because {@link + * RowCombiningTimeAndDimsIterator} doesn't need this method to return anything more specific than {@link + * TimeAndDimsPointer}. + */ + TimeAndDimsPointer getMarkedPointer(); +} diff --git a/processing/src/main/java/io/druid/segment/column/Column.java b/processing/src/main/java/io/druid/segment/column/Column.java index 6fe9f39d82c9..bc40ac8488d0 100644 --- a/processing/src/main/java/io/druid/segment/column/Column.java +++ b/processing/src/main/java/io/druid/segment/column/Column.java @@ -20,6 +20,7 @@ package io.druid.segment.column; import io.druid.java.util.common.StringUtils; +import io.druid.segment.selector.settable.SettableColumnValueSelector; /** */ @@ -43,4 +44,9 @@ static boolean storeDoubleAsFloat() ComplexColumn getComplexColumn(); BitmapIndex getBitmapIndex(); SpatialIndex getSpatialIndex(); + + /** + * Returns a new instance of a {@link SettableColumnValueSelector}, corresponding to the type of this column. + */ + SettableColumnValueSelector makeSettableColumnValueSelector(); } diff --git a/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java b/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java index 8cab3a11aa9d..70fe580336d2 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java @@ -30,5 +30,4 @@ public interface ColumnCapabilities boolean hasBitmapIndexes(); boolean hasSpatialIndexes(); boolean hasMultipleValues(); - } diff --git a/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java b/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java index cd5a1872ddb5..6179381dc6af 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java @@ -105,10 +105,10 @@ public ColumnCapabilitiesImpl setHasMultipleValues(boolean hasMultipleValues) return this; } - public ColumnCapabilitiesImpl merge(ColumnCapabilities other) + public void merge(ColumnCapabilities other) { if (other == null) { - return this; + return; } if (type == null) { @@ -124,7 +124,5 @@ public ColumnCapabilitiesImpl merge(ColumnCapabilities other) this.hasInvertedIndexes |= other.hasBitmapIndexes(); this.hasSpatialIndexes |= other.hasSpatialIndexes(); this.hasMultipleValues |= other.hasMultipleValues(); - - return this; } } diff --git a/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java b/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java index 51b54e07cdd5..72eef34e2a58 100644 --- a/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java @@ -31,6 +31,7 @@ */ public interface DictionaryEncodedColumn extends BaseColumn { + Class getClazz(); int length(); boolean hasMultipleValues(); int getSingleValueRow(int rowNum); diff --git a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java index 4d06634d908e..6d00d2ff0684 100644 --- a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java +++ b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java @@ -27,6 +27,8 @@ import io.druid.segment.data.ReadableOffset; +/** + */ public class DoublesColumn implements GenericColumn { /** @@ -57,14 +59,7 @@ public int length() @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - return column.makeColumnValueSelector(offset, IndexIO.LEGACY_FACTORY.getBitmapFactory() - .makeEmptyImmutableBitmap()); - } - - @Override - public float getFloatSingleValueRow(int rowNum) - { - return (float) column.get(rowNum); + return column.makeColumnValueSelector(offset, IndexIO.LEGACY_FACTORY.getBitmapFactory().makeEmptyImmutableBitmap()); } @Override @@ -73,18 +68,6 @@ public long getLongSingleValueRow(int rowNum) return (long) column.get(rowNum); } - @Override - public double getDoubleSingleValueRow(int rowNum) - { - return column.get(rowNum); - } - - @Override - public boolean isNull(int rowNum) - { - return false; - } - @Override public void close() { diff --git a/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java index dca4cb57cc41..5b6d34c8651c 100644 --- a/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java +++ b/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java @@ -45,9 +45,10 @@ public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) } @Override - public boolean isNull(int rowNum) + public long getLongSingleValueRow(int rowNum) { - return nullValueBitmap.get(rowNum); + assert !nullValueBitmap.get(rowNum); + return super.getLongSingleValueRow(rowNum); } @Override @@ -56,25 +57,4 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) super.inspectRuntimeShape(inspector); inspector.visit("nullValueBitmap", nullValueBitmap); } - - @Override - public float getFloatSingleValueRow(int rowNum) - { - assert !isNull(rowNum); - return super.getFloatSingleValueRow(rowNum); - } - - @Override - public long getLongSingleValueRow(int rowNum) - { - assert !isNull(rowNum); - return super.getLongSingleValueRow(rowNum); - } - - @Override - public double getDoubleSingleValueRow(int rowNum) - { - assert !isNull(rowNum); - return super.getLongSingleValueRow(rowNum); - } } diff --git a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java index cdf18d38a4b3..8826afb1dc00 100644 --- a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java @@ -58,14 +58,7 @@ public int length() @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - return column.makeColumnValueSelector(offset, IndexIO.LEGACY_FACTORY.getBitmapFactory() - .makeEmptyImmutableBitmap()); - } - - @Override - public float getFloatSingleValueRow(int rowNum) - { - return column.get(rowNum); + return column.makeColumnValueSelector(offset, IndexIO.LEGACY_FACTORY.getBitmapFactory().makeEmptyImmutableBitmap()); } @Override @@ -74,18 +67,6 @@ public long getLongSingleValueRow(int rowNum) return (long) column.get(rowNum); } - @Override - public double getDoubleSingleValueRow(int rowNum) - { - return (double) column.get(rowNum); - } - - @Override - public boolean isNull(int rowNum) - { - return false; - } - @Override public void close() { diff --git a/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java index 1d8e498ce3e7..0edb6a4576d5 100644 --- a/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java +++ b/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java @@ -45,9 +45,10 @@ public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) } @Override - public boolean isNull(int rowNum) + public long getLongSingleValueRow(int rowNum) { - return nullValueBitmap.get(rowNum); + assert !nullValueBitmap.get(rowNum); + return super.getLongSingleValueRow(rowNum); } @Override @@ -56,25 +57,4 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) super.inspectRuntimeShape(inspector); inspector.visit("nullValueBitmap", nullValueBitmap); } - - @Override - public float getFloatSingleValueRow(int rowNum) - { - assert !isNull(rowNum); - return super.getFloatSingleValueRow(rowNum); - } - - @Override - public long getLongSingleValueRow(int rowNum) - { - assert !isNull(rowNum); - return super.getLongSingleValueRow(rowNum); - } - - @Override - public double getDoubleSingleValueRow(int rowNum) - { - assert !isNull(rowNum); - return super.getLongSingleValueRow(rowNum); - } } diff --git a/processing/src/main/java/io/druid/segment/column/GenericColumn.java b/processing/src/main/java/io/druid/segment/column/GenericColumn.java index c00a3054e9ad..5e09259a3442 100644 --- a/processing/src/main/java/io/druid/segment/column/GenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/GenericColumn.java @@ -28,18 +28,9 @@ public interface GenericColumn extends BaseColumn, HotLoopCallee { int length(); - @CalledFromHotLoop - float getFloatSingleValueRow(int rowNum); - @CalledFromHotLoop long getLongSingleValueRow(int rowNum); - @CalledFromHotLoop - double getDoubleSingleValueRow(int rowNum); - - @CalledFromHotLoop - boolean isNull(int rowNum); - @Override void close(); } diff --git a/processing/src/main/java/io/druid/segment/column/IndexedComplexColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedComplexColumn.java index bd6076e2b1fe..daf0eb3a064f 100644 --- a/processing/src/main/java/io/druid/segment/column/IndexedComplexColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedComplexColumn.java @@ -28,13 +28,12 @@ public class IndexedComplexColumn implements ComplexColumn private final Indexed column; private final String typeName; - public IndexedComplexColumn( - String typeName, Indexed column - ) + public IndexedComplexColumn(String typeName, Indexed column) { this.column = column; this.typeName = typeName; } + @Override public Class getClazz() { diff --git a/processing/src/main/java/io/druid/segment/column/LongsColumn.java b/processing/src/main/java/io/druid/segment/column/LongsColumn.java index 0329c8aae54c..e41cfb1816c0 100644 --- a/processing/src/main/java/io/druid/segment/column/LongsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/LongsColumn.java @@ -62,30 +62,12 @@ public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) .makeEmptyImmutableBitmap()); } - @Override - public float getFloatSingleValueRow(int rowNum) - { - return (float) column.get(rowNum); - } - @Override public long getLongSingleValueRow(int rowNum) { return column.get(rowNum); } - @Override - public double getDoubleSingleValueRow(int rowNum) - { - return (double) column.get(rowNum); - } - - @Override - public boolean isNull(int rowNum) - { - return false; - } - @Override public void close() { diff --git a/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java index cd6689df7121..7e8e642b191c 100644 --- a/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java +++ b/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java @@ -45,9 +45,10 @@ public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) } @Override - public boolean isNull(int rowNum) + public long getLongSingleValueRow(int rowNum) { - return nullValueBitmap.get(rowNum); + assert !nullValueBitmap.get(rowNum); + return super.getLongSingleValueRow(rowNum); } @Override @@ -56,25 +57,4 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) super.inspectRuntimeShape(inspector); inspector.visit("nullValueBitmap", nullValueBitmap); } - - @Override - public float getFloatSingleValueRow(int rowNum) - { - assert !isNull(rowNum); - return super.getFloatSingleValueRow(rowNum); - } - - @Override - public long getLongSingleValueRow(int rowNum) - { - assert !isNull(rowNum); - return super.getLongSingleValueRow(rowNum); - } - - @Override - public double getDoubleSingleValueRow(int rowNum) - { - assert !isNull(rowNum); - return super.getLongSingleValueRow(rowNum); - } } diff --git a/processing/src/main/java/io/druid/segment/column/SimpleColumn.java b/processing/src/main/java/io/druid/segment/column/SimpleColumn.java index d4f86b44f938..6876a123bbdf 100644 --- a/processing/src/main/java/io/druid/segment/column/SimpleColumn.java +++ b/processing/src/main/java/io/druid/segment/column/SimpleColumn.java @@ -20,6 +20,7 @@ package io.druid.segment.column; import com.google.common.base.Supplier; +import io.druid.segment.selector.settable.SettableColumnValueSelector; /** */ @@ -92,4 +93,10 @@ public SpatialIndex getSpatialIndex() { return spatialIndex == null ? null : spatialIndex.get(); } + + @Override + public SettableColumnValueSelector makeSettableColumnValueSelector() + { + return getCapabilities().getType().makeSettableColumnValueSelector(); + } } diff --git a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java index 1cdf1ccbd244..4cd57470f753 100644 --- a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java @@ -60,6 +60,12 @@ public SimpleDictionaryEncodedColumn( this.cachedLookups = cachedLookups; } + @Override + public Class getClazz() + { + return String.class; + } + @Override public int length() { diff --git a/processing/src/main/java/io/druid/segment/column/ValueType.java b/processing/src/main/java/io/druid/segment/column/ValueType.java index d7154eaaf0b7..d22a8e98d32e 100644 --- a/processing/src/main/java/io/druid/segment/column/ValueType.java +++ b/processing/src/main/java/io/druid/segment/column/ValueType.java @@ -27,6 +27,12 @@ import io.druid.segment.DoubleWrappingDimensionSelector; import io.druid.segment.FloatWrappingDimensionSelector; import io.druid.segment.LongWrappingDimensionSelector; +import io.druid.segment.selector.settable.SettableColumnValueSelector; +import io.druid.segment.selector.settable.SettableDimensionValueSelector; +import io.druid.segment.selector.settable.SettableDoubleColumnValueSelector; +import io.druid.segment.selector.settable.SettableFloatColumnValueSelector; +import io.druid.segment.selector.settable.SettableLongColumnValueSelector; +import io.druid.segment.selector.settable.SettableObjectColumnValueSelector; /** * Should be the same as {@link io.druid.data.input.impl.DimensionSchema.ValueType}. @@ -43,6 +49,12 @@ public DimensionSelector makeNumericWrappingDimensionSelector( { return new FloatWrappingDimensionSelector(numericColumnValueSelector, extractionFn); } + + @Override + public SettableColumnValueSelector makeSettableColumnValueSelector() + { + return new SettableFloatColumnValueSelector(); + } }, DOUBLE { @Override @@ -53,6 +65,12 @@ public DimensionSelector makeNumericWrappingDimensionSelector( { return new DoubleWrappingDimensionSelector(numericColumnValueSelector, extractionFn); } + + @Override + public SettableColumnValueSelector makeSettableColumnValueSelector() + { + return new SettableDoubleColumnValueSelector(); + } }, LONG { @Override @@ -63,9 +81,27 @@ public DimensionSelector makeNumericWrappingDimensionSelector( { return new LongWrappingDimensionSelector(numericColumnValueSelector, extractionFn); } + + @Override + public SettableColumnValueSelector makeSettableColumnValueSelector() + { + return new SettableLongColumnValueSelector(); + } + }, + STRING { + @Override + public SettableColumnValueSelector makeSettableColumnValueSelector() + { + return new SettableDimensionValueSelector(); + } }, - STRING, - COMPLEX; + COMPLEX { + @Override + public SettableColumnValueSelector makeSettableColumnValueSelector() + { + return new SettableObjectColumnValueSelector(); + } + }; public DimensionSelector makeNumericWrappingDimensionSelector( ColumnValueSelector numericColumnValueSelector, @@ -75,6 +111,8 @@ public DimensionSelector makeNumericWrappingDimensionSelector( throw new UnsupportedOperationException("Not a numeric value type: " + name()); } + public abstract SettableColumnValueSelector makeSettableColumnValueSelector(); + public boolean isNumeric() { return isNumeric(this); diff --git a/processing/src/main/java/io/druid/segment/data/ArrayBasedIndexedInts.java b/processing/src/main/java/io/druid/segment/data/ArrayBasedIndexedInts.java index 608d6f968a71..2dcdb614c8fb 100644 --- a/processing/src/main/java/io/druid/segment/data/ArrayBasedIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/ArrayBasedIndexedInts.java @@ -27,6 +27,8 @@ */ public final class ArrayBasedIndexedInts implements IndexedInts { + static final IndexedInts EMPTY = new ArrayBasedIndexedInts(); + private int[] expansion; private int size; diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarIntsSerializer.java b/processing/src/main/java/io/druid/segment/data/ColumnarIntsSerializer.java index 770a9a8ee9d6..0072fb19d5bc 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarIntsSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarIntsSerializer.java @@ -29,6 +29,4 @@ public interface ColumnarIntsSerializer extends Serializer { void open() throws IOException; - - void add(Object obj) throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarMultiIntsSerializer.java b/processing/src/main/java/io/druid/segment/data/ColumnarMultiIntsSerializer.java index 8c8ec64f7d21..3c4e7f7c73f5 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarMultiIntsSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarMultiIntsSerializer.java @@ -19,10 +19,6 @@ package io.druid.segment.data; -import io.druid.java.util.common.IAE; -import it.unimi.dsi.fastutil.ints.IntArrayList; -import it.unimi.dsi.fastutil.ints.IntList; - import java.io.IOException; /** @@ -30,19 +26,5 @@ */ public abstract class ColumnarMultiIntsSerializer implements ColumnarIntsSerializer { - @Override - public void add(Object obj) throws IOException - { - if (obj == null) { - addValues(null); - } else if (obj instanceof int[]) { - addValues(IntArrayList.wrap((int[]) obj)); - } else if (obj instanceof IntList) { - addValues((IntList) obj); - } else { - throw new IAE("unsupported multi-value type: " + obj.getClass()); - } - } - - protected abstract void addValues(IntList vals) throws IOException; + public abstract void addValues(IndexedInts ints) throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedColumnarIntsSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedColumnarIntsSerializer.java index c27f8368e466..a98c1ec4c1ba 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedColumnarIntsSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedColumnarIntsSerializer.java @@ -94,7 +94,7 @@ public void open() throws IOException } @Override - protected void addValue(int val) throws IOException + public void addValue(int val) throws IOException { if (endBuffer == null) { throw new IllegalStateException("written out already"); diff --git a/processing/src/main/java/io/druid/segment/data/CompressedColumnarIntsSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedColumnarIntsSupplier.java index 23da0702348b..ad74167ee937 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedColumnarIntsSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedColumnarIntsSupplier.java @@ -109,7 +109,7 @@ public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws I } @VisibleForTesting - GenericIndexed getBaseIntBuffers() + GenericIndexed> getBaseIntBuffers() { return baseIntBuffers; } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeColumnarIntsSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeColumnarIntsSerializer.java index 20197031f2af..4f76063efa40 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeColumnarIntsSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeColumnarIntsSerializer.java @@ -129,7 +129,7 @@ public void open() throws IOException } @Override - protected void addValue(int val) throws IOException + public void addValue(int val) throws IOException { if (endBuffer == null) { throw new IllegalStateException("written out already"); diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeColumnarIntsSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeColumnarIntsSupplier.java index 1452fba08fcc..0cc58fedbf7e 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeColumnarIntsSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeColumnarIntsSupplier.java @@ -35,8 +35,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.IntBuffer; -import java.nio.ShortBuffer; import java.nio.channels.WritableByteChannel; import java.util.Iterator; @@ -255,48 +253,30 @@ public void remove() private class CompressedFullSizeColumnarInts extends CompressedVSizeColumnarInts { - IntBuffer intBuffer; - @Override - protected void loadBuffer(int bufferNum) + protected int _get(ByteBuffer buffer, boolean bigEngian, int bufferIndex) { - super.loadBuffer(bufferNum); - intBuffer = buffer.asIntBuffer(); - } - - @Override - protected int _get(int index) - { - return intBuffer.get(index); + return buffer.getInt(bufferIndex * Integer.BYTES); } } private class CompressedShortSizeColumnarInts extends CompressedVSizeColumnarInts { - ShortBuffer shortBuffer; - @Override - protected void loadBuffer(int bufferNum) - { - super.loadBuffer(bufferNum); - shortBuffer = buffer.asShortBuffer(); - } - - @Override - protected int _get(int index) + protected int _get(ByteBuffer buffer, boolean bigEngian, int bufferIndex) { // removes the need for padding - return shortBuffer.get(index) & 0xFFFF; + return buffer.getShort(bufferIndex * Short.BYTES) & 0xFFFF; } } private class CompressedByteSizeColumnarInts extends CompressedVSizeColumnarInts { @Override - protected int _get(int index) + protected int _get(ByteBuffer buffer, boolean bigEngian, int bufferIndex) { // removes the need for padding - return buffer.get(index) & 0xFF; + return buffer.get(bufferIndex) & 0xFF; } } @@ -333,24 +313,25 @@ public int get(int index) { // assumes the number of entries in each buffer is a power of 2 final int bufferNum = index >> div; + final int bufferIndex = index & rem; if (bufferNum != currBufferNum) { loadBuffer(bufferNum); } - return _get(index & rem); + return _get(buffer, bigEndian, bufferIndex); } /** - * Returns the value at the given index in the current decompression buffer + * Returns the value at the given bufferIndex in the current decompression buffer * - * @param index index of the value in the current buffer + * @param bufferIndex index of the value in the current buffer * - * @return the value at the given index + * @return the value at the given bufferIndex */ - protected int _get(final int index) + int _get(ByteBuffer buffer, boolean bigEndian, final int bufferIndex) { - final int pos = index * numBytes; + final int pos = bufferIndex * numBytes; // example for numBytes = 3 // big-endian: 0x000c0b0a stored 0c 0b 0a XX, read 0x0c0b0aXX >>> 8 // little-endian: 0x000c0b0a stored 0a 0b 0c XX, read 0xXX0c0b0a & 0x00FFFFFF @@ -382,7 +363,7 @@ public void close() @Override public String toString() { - return "CompressedVSizedIntsIndexedSupplier{" + + return "CompressedVSizeColumnarInts{" + "currBufferNum=" + currBufferNum + ", sizePer=" + sizePer + ", numChunks=" + singleThreadedBuffers.size() + diff --git a/processing/src/main/java/io/druid/segment/data/EntireLayoutColumnarDoublesSupplier.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutColumnarDoublesSupplier.java index eb0310e9d2b6..b3dc157e61fe 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutColumnarDoublesSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutColumnarDoublesSupplier.java @@ -44,6 +44,7 @@ public ColumnarDoubles get() private class EntireLayoutColumnarDoubles implements ColumnarDoubles { + @Override public int size() { @@ -65,7 +66,7 @@ public void close() @Override public String toString() { - return "EntireCompressedColumnarDoubles_Anonymous{" + + return "EntireLayoutColumnarDoubles{" + ", totalSize=" + totalSize + '}'; } diff --git a/processing/src/main/java/io/druid/segment/data/EntireLayoutColumnarFloatsSupplier.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutColumnarFloatsSupplier.java index b788f80461d2..8704e80564ea 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutColumnarFloatsSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutColumnarFloatsSupplier.java @@ -76,7 +76,7 @@ public void fill(int index, float[] toFill) @Override public String toString() { - return "EntireCompressedColumnarFloats_Anonymous{" + + return "EntireLayoutColumnarFloats{" + ", totalSize=" + totalSize + '}'; } diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 6dbe07f30f29..92160cdec714 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -398,6 +398,21 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) inspector.visit("strategy", strategy); } + @Override + public String toString() + { + StringBuilder sb = new StringBuilder("GenericIndexed["); + if (size() > 0) { + for (int i = 0; i < size(); i++) { + T value = get(i); + sb.append(value).append(',').append(' '); + } + sb.setLength(sb.length() - 2); + } + sb.append(']'); + return sb.toString(); + } + abstract class BufferIndexed implements Indexed { int lastReadSize; diff --git a/processing/src/main/java/io/druid/segment/data/IndexedInts.java b/processing/src/main/java/io/druid/segment/data/IndexedInts.java index 76ea0e619692..06e940cb70c1 100644 --- a/processing/src/main/java/io/druid/segment/data/IndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedInts.java @@ -34,6 +34,11 @@ @PublicApi public interface IndexedInts extends HotLoopCallee { + static IndexedInts empty() + { + return ArrayBasedIndexedInts.EMPTY; + } + @CalledFromHotLoop int size(); @CalledFromHotLoop diff --git a/processing/src/main/java/io/druid/segment/data/Offset.java b/processing/src/main/java/io/druid/segment/data/Offset.java index dacf0b7cfb42..0d081ad3ff2a 100644 --- a/processing/src/main/java/io/druid/segment/data/Offset.java +++ b/processing/src/main/java/io/druid/segment/data/Offset.java @@ -36,7 +36,7 @@ * * This interface is the core "pointer" interface that is used to create {@link io.druid.segment.ColumnValueSelector}s * over historical segments. It's counterpart for incremental index is {@link - * io.druid.segment.incremental.TimeAndDimsHolder}. + * io.druid.segment.incremental.IncrementalIndexRowHolder}. */ @SubclassesMustBePublic public abstract class Offset implements ReadableOffset, Cloneable diff --git a/processing/src/main/java/io/druid/segment/data/SingleValueColumnarIntsSerializer.java b/processing/src/main/java/io/druid/segment/data/SingleValueColumnarIntsSerializer.java index 8d2e3ef8af81..31eb82469d45 100644 --- a/processing/src/main/java/io/druid/segment/data/SingleValueColumnarIntsSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/SingleValueColumnarIntsSerializer.java @@ -19,8 +19,6 @@ package io.druid.segment.data; -import io.druid.java.util.common.IAE; - import java.io.IOException; /** @@ -28,24 +26,5 @@ */ public abstract class SingleValueColumnarIntsSerializer implements ColumnarIntsSerializer { - @Override - public void add(Object obj) throws IOException - { - if (obj == null) { - addValue(0); - } else if (obj instanceof Integer) { - addValue(((Number) obj).intValue()); - } else if (obj instanceof int[]) { - int[] vals = (int[]) obj; - if (vals.length == 0) { - addValue(0); - } else { - addValue(vals[0]); - } - } else { - throw new IAE("Unsupported single value type: " + obj.getClass()); - } - } - - protected abstract void addValue(int val) throws IOException; + public abstract void addValue(int val) throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializer.java b/processing/src/main/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializer.java index 2852a0b9e7e4..b3ccd6abdc5b 100644 --- a/processing/src/main/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializer.java @@ -17,15 +17,12 @@ * under the License. */ - package io.druid.segment.data; import io.druid.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.IndexIO; -import it.unimi.dsi.fastutil.ints.IntList; -import it.unimi.dsi.fastutil.ints.IntLists; +import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; import java.nio.ByteBuffer; @@ -87,19 +84,17 @@ public void open() throws IOException } @Override - protected void addValues(IntList vals) throws IOException + public void addValues(IndexedInts ints) throws IOException { if (lastOffsetWritten) { throw new IllegalStateException("written out already"); } - if (vals == null) { - vals = IntLists.EMPTY_LIST; - } - offsetWriter.add(offset); - for (int i = 0; i < vals.size(); i++) { - valueWriter.add(vals.getInt(i)); + offsetWriter.addValue(offset); + int numValues = ints.size(); + for (int i = 0; i < numValues; i++) { + valueWriter.addValue(ints.get(i)); } - offset += vals.size(); + offset += numValues; } @Override @@ -121,7 +116,7 @@ public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws I private void writeLastOffset() throws IOException { if (!lastOffsetWritten) { - offsetWriter.add(offset); + offsetWriter.addValue(offset); lastOffsetWritten = true; } } diff --git a/processing/src/main/java/io/druid/segment/data/VSizeColumnarInts.java b/processing/src/main/java/io/druid/segment/data/VSizeColumnarInts.java index ef7c4c2976eb..9ebf43f94ac0 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeColumnarInts.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeColumnarInts.java @@ -27,8 +27,6 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.serde.MetaSerdeHelper; import io.druid.segment.writeout.HeapByteBufferWriteOutBytes; -import it.unimi.dsi.fastutil.ints.IntArrayList; -import it.unimi.dsi.fastutil.ints.IntList; import java.io.IOException; import java.nio.ByteBuffer; @@ -52,24 +50,24 @@ public static VSizeColumnarInts fromArray(int[] array) public static VSizeColumnarInts fromArray(int[] array, int maxValue) { - return fromList(IntArrayList.wrap(array), maxValue); + return fromIndexedInts(new ArrayBasedIndexedInts(array), maxValue); } - public static VSizeColumnarInts fromList(IntList list, int maxValue) + public static VSizeColumnarInts fromIndexedInts(IndexedInts ints, int maxValue) { int numBytes = getNumBytesForMax(maxValue); - final ByteBuffer buffer = ByteBuffer.allocate((list.size() * numBytes) + (4 - numBytes)); - writeToBuffer(buffer, list, numBytes, maxValue); + final ByteBuffer buffer = ByteBuffer.allocate((ints.size() * numBytes) + (4 - numBytes)); + writeToBuffer(buffer, ints, numBytes, maxValue); return new VSizeColumnarInts(buffer.asReadOnlyBuffer(), numBytes); } - private static void writeToBuffer(ByteBuffer buffer, IntList list, int numBytes, int maxValue) + private static void writeToBuffer(ByteBuffer buffer, IndexedInts ints, int numBytes, int maxValue) { ByteBuffer helperBuffer = ByteBuffer.allocate(Integer.BYTES); - for (int i = 0; i < list.size(); i++) { - int val = list.getInt(i); + for (int i = 0, size = ints.size(); i < size; i++) { + int val = ints.get(i); if (val < 0) { throw new IAE("integer values must be positive, got[%d], i[%d]", val, i); } diff --git a/processing/src/main/java/io/druid/segment/data/VSizeColumnarIntsSerializer.java b/processing/src/main/java/io/druid/segment/data/VSizeColumnarIntsSerializer.java index 3958c646f584..5c3b944be58c 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeColumnarIntsSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeColumnarIntsSerializer.java @@ -62,7 +62,7 @@ public void open() throws IOException } @Override - protected void addValue(int val) throws IOException + public void addValue(int val) throws IOException { if (bufPaddingWritten) { throw new IllegalStateException("written out already"); diff --git a/processing/src/main/java/io/druid/segment/data/VSizeColumnarMultiIntsSerializer.java b/processing/src/main/java/io/druid/segment/data/VSizeColumnarMultiIntsSerializer.java index 42209456de99..02a37fc2b941 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeColumnarMultiIntsSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeColumnarMultiIntsSerializer.java @@ -22,10 +22,9 @@ import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.segment.writeout.WriteOutBytes; -import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.serde.MetaSerdeHelper; -import it.unimi.dsi.fastutil.ints.IntList; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.writeout.WriteOutBytes; import java.io.IOException; import java.nio.channels.WritableByteChannel; @@ -104,19 +103,18 @@ public void open() throws IOException } @Override - protected void addValues(IntList ints) throws IOException + public void addValues(IndexedInts ints) throws IOException { if (numBytesForMaxWritten) { throw new IllegalStateException("written out already"); } - if (ints != null) { - for (int i = 0; i < ints.size(); i++) { - int value = ints.getInt(i); - Preconditions.checkState(value >= 0 && value <= maxId); - writeInt.write(valuesOut, value); - } + for (int i = 0, size = ints.size(); i < size; i++) { + int value = ints.get(i); + Preconditions.checkState(value >= 0 && value <= maxId); + writeInt.write(valuesOut, value); } headerOut.writeInt(Ints.checkedCast(valuesOut.size())); + ++numWritten; } diff --git a/processing/src/main/java/io/druid/segment/filter/Filters.java b/processing/src/main/java/io/druid/segment/filter/Filters.java index 684fd2f28fb5..415ab5bdc7dc 100644 --- a/processing/src/main/java/io/druid/segment/filter/Filters.java +++ b/processing/src/main/java/io/druid/segment/filter/Filters.java @@ -55,6 +55,7 @@ import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntList; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -104,7 +105,8 @@ public Filter apply(DimFilter input) * * @return converted filter, or null if input was null */ - public static Filter toFilter(DimFilter dimFilter) + @Nullable + public static Filter toFilter(@Nullable DimFilter dimFilter) { return dimFilter == null ? null : dimFilter.toFilter(); } @@ -468,7 +470,8 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - public static Filter convertToCNFFromQueryContext(Query query, Filter filter) + @Nullable + public static Filter convertToCNFFromQueryContext(Query query, @Nullable Filter filter) { if (filter == null) { return null; diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index ecfd6b95a7a9..3561df553db6 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -78,11 +78,8 @@ import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import java.io.Closeable; -import java.lang.reflect.Array; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.Deque; import java.util.Iterator; @@ -458,7 +455,7 @@ protected abstract AddToFactsResult addToFacts( boolean reportParseExceptions, InputRow row, AtomicInteger numEntries, - TimeAndDims key, + IncrementalIndexRow key, ThreadLocal rowContainer, Supplier rowSupplier, boolean skipMaxRowsInMemoryCheck @@ -480,51 +477,45 @@ protected abstract AddToFactsResult addToFacts( protected abstract boolean isNull(int rowOffset, int aggOffset); - public static class TimeAndDimsResult + static class IncrementalIndexRowResult { - private TimeAndDims timeAndDims; + private IncrementalIndexRow incrementalIndexRow; private List parseExceptionMessages; - public TimeAndDimsResult( - TimeAndDims timeAndDims, - List parseExceptionMessages - ) + IncrementalIndexRowResult(IncrementalIndexRow incrementalIndexRow, List parseExceptionMessages) { - this.timeAndDims = timeAndDims; + this.incrementalIndexRow = incrementalIndexRow; this.parseExceptionMessages = parseExceptionMessages; } - public TimeAndDims getTimeAndDims() + IncrementalIndexRow getIncrementalIndexRow() { - return timeAndDims; + return incrementalIndexRow; } - public List getParseExceptionMessages() + List getParseExceptionMessages() { return parseExceptionMessages; } } - public static class AddToFactsResult + static class AddToFactsResult { private int rowCount; private List parseExceptionMessages; - public AddToFactsResult( - int rowCount, - List parseExceptionMessages - ) + AddToFactsResult(int rowCount, List parseExceptionMessages) { this.rowCount = rowCount; this.parseExceptionMessages = parseExceptionMessages; } - public int getRowCount() + int getRowCount() { return rowCount; } - public List getParseExceptionMessages() + List getParseExceptionMessages() { return parseExceptionMessages; } @@ -571,14 +562,14 @@ public IncrementalIndexAddResult add(InputRow row) throws IndexSizeExceededExcep public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException { - TimeAndDimsResult timeAndDimsResult = toTimeAndDims(row); + IncrementalIndexRowResult incrementalIndexRowResult = toIncrementalIndexRow(row); final AddToFactsResult addToFactsResult = addToFacts( metrics, deserializeComplexMetrics, reportParseExceptions, row, numEntries, - timeAndDimsResult.getTimeAndDims(), + incrementalIndexRowResult.getIncrementalIndexRow(), in, rowSupplier, skipMaxRowsInMemoryCheck @@ -586,14 +577,14 @@ public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCh updateMaxIngestedTime(row.getTimestamp()); ParseException parseException = getCombinedParseException( row, - timeAndDimsResult.getParseExceptionMessages(), + incrementalIndexRowResult.getParseExceptionMessages(), addToFactsResult.getParseExceptionMessages() ); return new IncrementalIndexAddResult(addToFactsResult.getRowCount(), parseException); } @VisibleForTesting - TimeAndDimsResult toTimeAndDims(InputRow row) + IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) { row = formatRow(row); if (row.getTimestampFromEpoch() < minTimestamp) { @@ -644,7 +635,9 @@ TimeAndDimsResult toTimeAndDims(InputRow row) } // Set column capabilities as data is coming in - if (!capabilities.hasMultipleValues() && dimsKey != null && handler.getLengthOfEncodedKeyComponent(dimsKey) > 1) { + if (!capabilities.hasMultipleValues() && + dimsKey != null && + handler.getLengthOfEncodedKeyComponent(dimsKey) > 1) { capabilities.setHasMultipleValues(true); } @@ -684,15 +677,18 @@ TimeAndDimsResult toTimeAndDims(InputRow row) if (row.getTimestamp() != null) { truncated = gran.bucketStart(row.getTimestamp()).getMillis(); } - - TimeAndDims timeAndDims = new TimeAndDims(Math.max(truncated, minTimestamp), dims, dimensionDescsList); - return new TimeAndDimsResult(timeAndDims, parseExceptionMessages); + IncrementalIndexRow incrementalIndexRow = new IncrementalIndexRow( + Math.max(truncated, minTimestamp), + dims, + dimensionDescsList + ); + return new IncrementalIndexRowResult(incrementalIndexRow, parseExceptionMessages); } public static ParseException getCombinedParseException( InputRow row, - List dimParseExceptionMessages, - List aggParseExceptionMessages + @Nullable List dimParseExceptionMessages, + @Nullable List aggParseExceptionMessages ) { int numAdded = 0; @@ -790,7 +786,7 @@ public String getMetricType(String metric) return metricDesc != null ? metricDesc.getType() : null; } - public ColumnValueSelector makeMetricColumnValueSelector(String metric, TimeAndDimsHolder currEntry) + public ColumnValueSelector makeMetricColumnValueSelector(String metric, IncrementalIndexRowHolder currEntry) { MetricDesc metricDesc = metricDescs.get(metric); if (metricDesc == null) { @@ -831,6 +827,7 @@ public DateTime getMaxTime() return isEmpty() ? null : DateTimes.utc(getMaxTimeMillis()); } + @Nullable public Integer getDimensionIndex(String dimension) { DimensionDesc dimSpec = getDimension(dimension); @@ -922,15 +919,6 @@ private static AggregatorFactory[] getCombiningAggregators(AggregatorFactory[] a return combiningAggregators; } - public Map getDimensionHandlers() - { - Map handlers = Maps.newLinkedHashMap(); - for (DimensionDesc desc : dimensionDescsList) { - handlers.put(desc.getName(), desc.getHandler()); - } - return handlers; - } - @Override public Iterator iterator() { @@ -948,10 +936,10 @@ public Iterator iterator() return Iterators.transform( getFacts().iterator(descending), - timeAndDims -> { - final int rowOffset = timeAndDims.getRowIndex(); + incrementalIndexRow -> { + final int rowOffset = incrementalIndexRow.getRowIndex(); - Object[] theDims = timeAndDims.getDims(); + Object[] theDims = incrementalIndexRow.getDims(); Map theVals = Maps.newLinkedHashMap(); for (int i = 0; i < theDims.length; ++i) { @@ -982,7 +970,7 @@ public Iterator iterator() } } - return new MapBasedRow(timeAndDims.getTimestamp(), theVals); + return new MapBasedRow(incrementalIndexRow.getTimestamp(), theVals); } ); } @@ -1087,123 +1075,6 @@ public ColumnCapabilitiesImpl getCapabilities() } } - public static final class TimeAndDims - { - public static final int EMPTY_ROW_INDEX = -1; - - private final long timestamp; - private final Object[] dims; - private final List dimensionDescsList; - - /** - * rowIndex is not checked in {@link #equals} and {@link #hashCode} on purpose. TimeAndDims acts as a Map key - * and "entry" object (rowIndex is the "value") at the same time. This is done to reduce object indirection and - * improve locality, and avoid boxing of rowIndex as Integer, when stored in JDK collection: - * {@link RollupFactsHolder} needs concurrent collections, that are not present in fastutil. - */ - private int rowIndex; - - TimeAndDims( - long timestamp, - Object[] dims, - List dimensionDescsList - ) - { - this(timestamp, dims, dimensionDescsList, EMPTY_ROW_INDEX); - } - - TimeAndDims( - long timestamp, - Object[] dims, - List dimensionDescsList, - int rowIndex - ) - { - this.timestamp = timestamp; - this.dims = dims; - this.dimensionDescsList = dimensionDescsList; - this.rowIndex = rowIndex; - } - - public long getTimestamp() - { - return timestamp; - } - - public Object[] getDims() - { - return dims; - } - - public int getRowIndex() - { - return rowIndex; - } - - private void setRowIndex(int rowIndex) - { - this.rowIndex = rowIndex; - } - - @Override - public String toString() - { - return "TimeAndDims{" + - "timestamp=" + DateTimes.utc(timestamp) + - ", dims=" + Lists.transform( - Arrays.asList(dims), new Function() - { - @Override - public Object apply(@Nullable Object input) - { - if (input == null || Array.getLength(input) == 0) { - return Collections.singletonList("null"); - } - return Collections.singletonList(input); - } - } - ) + '}'; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - TimeAndDims that = (TimeAndDims) o; - - if (timestamp != that.timestamp) { - return false; - } - if (dims.length != that.dims.length) { - return false; - } - for (int i = 0; i < dims.length; i++) { - final DimensionIndexer indexer = dimensionDescsList.get(i).getIndexer(); - if (!indexer.checkUnsortedEncodedKeyComponentsEqual(dims[i], that.dims[i])) { - return false; - } - } - return true; - } - - @Override - public int hashCode() - { - int hash = (int) timestamp; - for (int i = 0; i < dims.length; i++) { - final DimensionIndexer indexer = dimensionDescsList.get(i).getIndexer(); - hash = 31 * hash + indexer.getUnsortedEncodedKeyComponentHashCode(dims[i]); - } - return hash; - } - } - protected ColumnSelectorFactory makeColumnSelectorFactory( final AggregatorFactory agg, final Supplier in, @@ -1213,23 +1084,23 @@ protected ColumnSelectorFactory makeColumnSelectorFactory( return makeColumnSelectorFactory(virtualColumns, agg, in, deserializeComplexMetrics); } - protected final Comparator dimsComparator() + protected final Comparator dimsComparator() { - return new TimeAndDimsComp(dimensionDescsList); + return new IncrementalIndexRowComparator(dimensionDescsList); } @VisibleForTesting - static final class TimeAndDimsComp implements Comparator + static final class IncrementalIndexRowComparator implements Comparator { private List dimensionDescs; - public TimeAndDimsComp(List dimDescs) + public IncrementalIndexRowComparator(List dimDescs) { this.dimensionDescs = dimDescs; } @Override - public int compare(TimeAndDims lhs, TimeAndDims rhs) + public int compare(IncrementalIndexRow lhs, IncrementalIndexRow rhs) { int retVal = Longs.compare(lhs.timestamp, rhs.timestamp); int numComparisons = Math.min(lhs.dims.length, rhs.dims.length); @@ -1283,25 +1154,25 @@ interface FactsHolder { /** * @return the previous rowIndex associated with the specified key, or - * {@code TimeAndDims#EMPTY_ROW_INDEX} if there was no mapping for the key. + * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. */ - int getPriorIndex(TimeAndDims key); + int getPriorIndex(IncrementalIndexRow key); long getMinTimeMillis(); long getMaxTimeMillis(); - Iterator iterator(boolean descending); + Iterator iterator(boolean descending); - Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd); + Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd); - Iterable keySet(); + Iterable keySet(); /** * @return the previous rowIndex associated with the specified key, or - * {@code TimeAndDims#EMPTY_ROW_INDEX} if there was no mapping for the key. + * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. */ - int putIfAbsent(TimeAndDims key, int rowIndex); + int putIfAbsent(IncrementalIndexRow key, int rowIndex); void clear(); } @@ -1310,14 +1181,18 @@ static class RollupFactsHolder implements FactsHolder { private final boolean sortFacts; // Can't use Set because we need to be able to get from collection - private final ConcurrentMap facts; + private final ConcurrentMap facts; private final List dimensionDescsList; - public RollupFactsHolder(boolean sortFacts, Comparator timeAndDimsComparator, List dimensionDescsList) + RollupFactsHolder( + boolean sortFacts, + Comparator incrementalIndexRowComparator, + List dimensionDescsList + ) { this.sortFacts = sortFacts; if (sortFacts) { - this.facts = new ConcurrentSkipListMap<>(timeAndDimsComparator); + this.facts = new ConcurrentSkipListMap<>(incrementalIndexRowComparator); } else { this.facts = new ConcurrentHashMap<>(); } @@ -1325,17 +1200,17 @@ public RollupFactsHolder(boolean sortFacts, Comparator timeAndDimsC } @Override - public int getPriorIndex(TimeAndDims key) + public int getPriorIndex(IncrementalIndexRow key) { - TimeAndDims timeAndDims = facts.get(key); - return timeAndDims == null ? TimeAndDims.EMPTY_ROW_INDEX : timeAndDims.rowIndex; + IncrementalIndexRow row = facts.get(key); + return row == null ? IncrementalIndexRow.EMPTY_ROW_INDEX : row.getRowIndex(); } @Override public long getMinTimeMillis() { if (sortFacts) { - return ((ConcurrentNavigableMap) facts).firstKey().getTimestamp(); + return ((ConcurrentNavigableMap) facts).firstKey().getTimestamp(); } else { throw new UnsupportedOperationException("can't get minTime from unsorted facts data."); } @@ -1345,48 +1220,48 @@ public long getMinTimeMillis() public long getMaxTimeMillis() { if (sortFacts) { - return ((ConcurrentNavigableMap) facts).lastKey().getTimestamp(); + return ((ConcurrentNavigableMap) facts).lastKey().getTimestamp(); } else { throw new UnsupportedOperationException("can't get maxTime from unsorted facts data."); } } @Override - public Iterator iterator(boolean descending) + public Iterator iterator(boolean descending) { if (descending && sortFacts) { - return ((ConcurrentNavigableMap) facts).descendingMap().keySet().iterator(); + return ((ConcurrentNavigableMap) facts).descendingMap().keySet().iterator(); } return keySet().iterator(); } @Override - public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) + public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) { if (!sortFacts) { throw new UnsupportedOperationException("can't get timeRange from unsorted facts data."); } - TimeAndDims start = new TimeAndDims(timeStart, new Object[]{}, dimensionDescsList); - TimeAndDims end = new TimeAndDims(timeEnd, new Object[]{}, dimensionDescsList); - ConcurrentNavigableMap subMap = - ((ConcurrentNavigableMap) facts).subMap(start, end); - final Map rangeMap = descending ? subMap.descendingMap() : subMap; + IncrementalIndexRow start = new IncrementalIndexRow(timeStart, new Object[]{}, dimensionDescsList); + IncrementalIndexRow end = new IncrementalIndexRow(timeEnd, new Object[]{}, dimensionDescsList); + ConcurrentNavigableMap subMap = + ((ConcurrentNavigableMap) facts).subMap(start, end); + final Map rangeMap = descending ? subMap.descendingMap() : subMap; return rangeMap.keySet(); } @Override - public Iterable keySet() + public Iterable keySet() { return facts.keySet(); } @Override - public int putIfAbsent(TimeAndDims key, int rowIndex) + public int putIfAbsent(IncrementalIndexRow key, int rowIndex) { // setRowIndex() must be called before facts.putIfAbsent() for visibility of rowIndex from concurrent readers. key.setRowIndex(rowIndex); - TimeAndDims prev = facts.putIfAbsent(key, key); - return prev == null ? TimeAndDims.EMPTY_ROW_INDEX : prev.rowIndex; + IncrementalIndexRow prev = facts.putIfAbsent(key, key); + return prev == null ? IncrementalIndexRow.EMPTY_ROW_INDEX : prev.getRowIndex(); } @Override @@ -1399,7 +1274,7 @@ public void clear() static class PlainFactsHolder implements FactsHolder { private final boolean sortFacts; - private final ConcurrentMap> facts; + private final ConcurrentMap> facts; public PlainFactsHolder(boolean sortFacts) { @@ -1412,17 +1287,17 @@ public PlainFactsHolder(boolean sortFacts) } @Override - public int getPriorIndex(TimeAndDims key) + public int getPriorIndex(IncrementalIndexRow key) { // always return EMPTY_ROW_INDEX to indicate that no prior key cause we always add new row - return TimeAndDims.EMPTY_ROW_INDEX; + return IncrementalIndexRow.EMPTY_ROW_INDEX; } @Override public long getMinTimeMillis() { if (sortFacts) { - return ((ConcurrentNavigableMap>) facts).firstKey(); + return ((ConcurrentNavigableMap>) facts).firstKey(); } else { throw new UnsupportedOperationException("can't get minTime from unsorted facts data."); } @@ -1432,33 +1307,33 @@ public long getMinTimeMillis() public long getMaxTimeMillis() { if (sortFacts) { - return ((ConcurrentNavigableMap>) facts).lastKey(); + return ((ConcurrentNavigableMap>) facts).lastKey(); } else { throw new UnsupportedOperationException("can't get maxTime from unsorted facts data."); } } @Override - public Iterator iterator(boolean descending) + public Iterator iterator(boolean descending) { if (descending && sortFacts) { - return concat(((ConcurrentNavigableMap>) facts) + return concat(((ConcurrentNavigableMap>) facts) .descendingMap().values(), true).iterator(); } return concat(facts.values(), false).iterator(); } @Override - public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) + public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) { - ConcurrentNavigableMap> subMap = - ((ConcurrentNavigableMap>) facts).subMap(timeStart, timeEnd); - final Map> rangeMap = descending ? subMap.descendingMap() : subMap; + ConcurrentNavigableMap> subMap = + ((ConcurrentNavigableMap>) facts).subMap(timeStart, timeEnd); + final Map> rangeMap = descending ? subMap.descendingMap() : subMap; return concat(rangeMap.values(), descending); } - private Iterable concat( - final Iterable> iterable, + private Iterable concat( + final Iterable> iterable, final boolean descending ) { @@ -1471,16 +1346,16 @@ private Iterable concat( } @Override - public Iterable keySet() + public Iterable keySet() { return concat(facts.values(), false); } @Override - public int putIfAbsent(TimeAndDims key, int rowIndex) + public int putIfAbsent(IncrementalIndexRow key, int rowIndex) { Long time = key.getTimestamp(); - Deque rows = facts.get(time); + Deque rows = facts.get(time); if (rows == null) { facts.putIfAbsent(time, new ConcurrentLinkedDeque<>()); // in race condition, rows may be put by other thread, so always get latest status from facts @@ -1490,7 +1365,7 @@ public int putIfAbsent(TimeAndDims key, int rowIndex) key.setRowIndex(rowIndex); rows.add(key); // always return EMPTY_ROW_INDEX to indicate that we always add new row - return TimeAndDims.EMPTY_ROW_INDEX; + return IncrementalIndexRow.EMPTY_ROW_INDEX; } @Override @@ -1502,10 +1377,10 @@ public void clear() private class LongMetricColumnSelector implements LongColumnSelector { - private final TimeAndDimsHolder currEntry; + private final IncrementalIndexRowHolder currEntry; private final int metricIndex; - public LongMetricColumnSelector(TimeAndDimsHolder currEntry, int metricIndex) + public LongMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) { this.currEntry = currEntry; this.metricIndex = metricIndex; @@ -1515,7 +1390,7 @@ public LongMetricColumnSelector(TimeAndDimsHolder currEntry, int metricIndex) public long getLong() { assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricLongValue(currEntry.getValue(), metricIndex); + return getMetricLongValue(currEntry.get().getRowIndex(), metricIndex); } @Override @@ -1527,19 +1402,19 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public boolean isNull() { - return IncrementalIndex.this.isNull(currEntry.getValue(), metricIndex); + return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); } } private class ObjectMetricColumnSelector extends ObjectColumnSelector { - private final TimeAndDimsHolder currEntry; + private final IncrementalIndexRowHolder currEntry; private final int metricIndex; private Class classOfObject; public ObjectMetricColumnSelector( MetricDesc metricDesc, - TimeAndDimsHolder currEntry, + IncrementalIndexRowHolder currEntry, int metricIndex ) { @@ -1552,7 +1427,7 @@ public ObjectMetricColumnSelector( @Override public Object getObject() { - return getMetricObjectValue(currEntry.getValue(), metricIndex); + return getMetricObjectValue(currEntry.get().getRowIndex(), metricIndex); } @Override @@ -1570,10 +1445,10 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) private class FloatMetricColumnSelector implements FloatColumnSelector { - private final TimeAndDimsHolder currEntry; + private final IncrementalIndexRowHolder currEntry; private final int metricIndex; - public FloatMetricColumnSelector(TimeAndDimsHolder currEntry, int metricIndex) + public FloatMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) { this.currEntry = currEntry; this.metricIndex = metricIndex; @@ -1583,7 +1458,7 @@ public FloatMetricColumnSelector(TimeAndDimsHolder currEntry, int metricIndex) public float getFloat() { assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricFloatValue(currEntry.getValue(), metricIndex); + return getMetricFloatValue(currEntry.get().getRowIndex(), metricIndex); } @Override @@ -1595,16 +1470,16 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public boolean isNull() { - return IncrementalIndex.this.isNull(currEntry.getValue(), metricIndex); + return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); } } private class DoubleMetricColumnSelector implements DoubleColumnSelector { - private final TimeAndDimsHolder currEntry; + private final IncrementalIndexRowHolder currEntry; private final int metricIndex; - public DoubleMetricColumnSelector(TimeAndDimsHolder currEntry, int metricIndex) + public DoubleMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) { this.currEntry = currEntry; this.metricIndex = metricIndex; @@ -1614,13 +1489,13 @@ public DoubleMetricColumnSelector(TimeAndDimsHolder currEntry, int metricIndex) public double getDouble() { assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricDoubleValue(currEntry.getValue(), metricIndex); + return getMetricDoubleValue(currEntry.get().getRowIndex(), metricIndex); } @Override public boolean isNull() { - return IncrementalIndex.this.isNull(currEntry.getValue(), metricIndex); + return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); } @Override diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java index 8eb148d6bb4a..78c1206706c9 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java @@ -19,27 +19,23 @@ package io.druid.segment.incremental; -import com.google.common.base.Function; -import com.google.common.collect.Iterators; -import com.google.common.collect.Maps; import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.MutableBitmap; -import io.druid.segment.DimensionHandler; import io.druid.segment.DimensionIndexer; import io.druid.segment.IndexableAdapter; import io.druid.segment.IntIteratorUtils; import io.druid.segment.Metadata; -import io.druid.segment.Rowboat; +import io.druid.segment.TransformableRowIterator; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.data.BitmapValues; import io.druid.segment.data.Indexed; -import io.druid.segment.data.ListIndexed; import it.unimi.dsi.fastutil.ints.IntIterator; import org.joda.time.Interval; -import java.util.Iterator; +import javax.annotation.Nullable; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** */ @@ -67,32 +63,38 @@ public DimensionAccessor(IncrementalIndex.DimensionDesc dimensionDesc) } } - public IncrementalIndexAdapter( - Interval dataInterval, IncrementalIndex index, BitmapFactory bitmapFactory - ) + public IncrementalIndexAdapter(Interval dataInterval, IncrementalIndex index, BitmapFactory bitmapFactory) { this.dataInterval = dataInterval; this.index = index; - /* Sometimes it's hard to tell whether one dimension contains a null value or not. - * If one dimension had show a null or empty value explicitly, then yes, it contains - * null value. But if one dimension's values are all non-null, it still early to say - * this dimension does not contain null value. Consider a two row case, first row had - * "dimA=1" and "dimB=2", the second row only had "dimA=3". To dimB, its value are "2" and - * never showed a null or empty value. But when we combines these two rows, dimB is null - * in row 2. So we should iterate all rows to determine whether one dimension contains - * a null value. - */ final List dimensions = index.getDimensions(); + accessors = dimensions + .stream() + .collect(Collectors.toMap(IncrementalIndex.DimensionDesc::getName, DimensionAccessor::new)); - accessors = Maps.newHashMapWithExpectedSize(dimensions.size()); - for (IncrementalIndex.DimensionDesc dimension : dimensions) { - accessors.put(dimension.getName(), new DimensionAccessor(dimension)); - } + processRows(index, bitmapFactory, dimensions); + } + /** + * Sometimes it's hard to tell whether one dimension contains a null value or not. + * If one dimension had show a null or empty value explicitly, then yes, it contains + * null value. But if one dimension's values are all non-null, it still early to say + * this dimension does not contain null value. Consider a two row case, first row had + * "dimA=1" and "dimB=2", the second row only had "dimA=3". To dimB, its value are "2" and + * never showed a null or empty value. But when we combines these two rows, dimB is null + * in row 2. So we should iterate all rows to determine whether one dimension contains + * a null value. + */ + private void processRows( + IncrementalIndex index, + BitmapFactory bitmapFactory, + List dimensions + ) + { int rowNum = 0; - for (IncrementalIndex.TimeAndDims timeAndDims : index.getFacts().keySet()) { - final Object[] dims = timeAndDims.getDims(); + for (IncrementalIndexRow row : index.getFacts().keySet()) { + final Object[] dims = row.getDims(); for (IncrementalIndex.DimensionDesc dimension : dimensions) { final int dimIndex = dimension.getIndex(); @@ -128,19 +130,20 @@ public int getNumRows() } @Override - public Indexed getDimensionNames() + public List getDimensionNames() { - return new ListIndexed(index.getDimensionNames(), String.class); + return index.getDimensionNames(); } @Override - public Indexed getMetricNames() + public List getMetricNames() { - return new ListIndexed(index.getMetricNames(), String.class); + return index.getMetricNames(); } + @Nullable @Override - public Indexed getDimValueLookup(String dimension) + public > Indexed getDimValueLookup(String dimension) { final DimensionAccessor accessor = accessors.get(dimension); if (accessor == null) { @@ -153,68 +156,9 @@ public Indexed getDimValueLookup(String dimension) } @Override - public Iterable getRows() + public TransformableRowIterator getRows() { - return new Iterable() - { - @Override - public Iterator iterator() - { - final List dimensions = index.getDimensions(); - final DimensionHandler[] handlers = new DimensionHandler[dimensions.size()]; - final DimensionIndexer[] indexers = new DimensionIndexer[dimensions.size()]; - for (IncrementalIndex.DimensionDesc dimension : dimensions) { - handlers[dimension.getIndex()] = dimension.getHandler(); - indexers[dimension.getIndex()] = dimension.getIndexer(); - } - - /* - * Note that the transform function increments a counter to determine the rowNum of - * the iterated Rowboats. We need to return a new iterator on each - * iterator() call to ensure the counter starts at 0. - */ - return Iterators.transform( - index.getFacts().keySet().iterator(), - new Function() - { - int count = 0; - - @Override - public Rowboat apply(IncrementalIndex.TimeAndDims timeAndDims) - { - final Object[] dimValues = timeAndDims.getDims(); - final int rowOffset = timeAndDims.getRowIndex(); - - Object[] dims = new Object[dimValues.length]; - for (IncrementalIndex.DimensionDesc dimension : dimensions) { - final int dimIndex = dimension.getIndex(); - - if (dimIndex >= dimValues.length || dimValues[dimIndex] == null) { - continue; - } - - final DimensionIndexer indexer = indexers[dimIndex]; - Object sortedDimVals = indexer.convertUnsortedEncodedKeyComponentToSortedEncodedKeyComponent(dimValues[dimIndex]); - dims[dimIndex] = sortedDimVals; - } - - Object[] metrics = new Object[index.getMetricAggs().length]; - for (int i = 0; i < metrics.length; i++) { - metrics[i] = index.getMetricObjectValue(rowOffset, i); - } - - return new Rowboat( - timeAndDims.getTimestamp(), - dims, - metrics, - count++, - handlers - ); - } - } - ); - } - }; + return new IncrementalIndexRowIterator(index); } @Override @@ -245,18 +189,6 @@ public BitmapValues getBitmapValues(String dimension, int index) return new MutableBitmapValues(bitmapIndex); } - @Override - public String getMetricType(String metric) - { - return index.getMetricType(metric); - } - - @Override - public ColumnCapabilities getCapabilities(String column) - { - return index.getCapabilities(column); - } - static class MutableBitmapValues implements BitmapValues { private final MutableBitmap bitmapIndex; @@ -280,14 +212,20 @@ public IntIterator iterator() } @Override - public Metadata getMetadata() + public String getMetricType(String metric) { - return index.getMetadata(); + return index.getMetricType(metric); } @Override - public Map getDimensionHandlers() + public ColumnCapabilities getCapabilities(String column) + { + return index.getCapabilities(column); + } + + @Override + public Metadata getMetadata() { - return index.getDimensionHandlers(); + return index.getMetadata(); } } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java index 23afec83166a..39119bbb2831 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java @@ -21,13 +21,11 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; -import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnValueSelector; import io.druid.segment.DimensionIndexer; import io.druid.segment.DimensionSelector; import io.druid.segment.DimensionSelectorUtils; -import io.druid.segment.LongColumnSelector; import io.druid.segment.SingleScanTimeDimSelector; import io.druid.segment.VirtualColumns; import io.druid.segment.column.Column; @@ -44,19 +42,19 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory private final IncrementalIndex index; private final VirtualColumns virtualColumns; private final boolean descending; - private final TimeAndDimsHolder timeAndDimsHolder; + private final IncrementalIndexRowHolder rowHolder; IncrementalIndexColumnSelectorFactory( IncrementalIndex index, VirtualColumns virtualColumns, boolean descending, - TimeAndDimsHolder timeAndDimsHolder + IncrementalIndexRowHolder rowHolder ) { this.index = index; this.virtualColumns = virtualColumns; this.descending = descending; - this.timeAndDimsHolder = timeAndDimsHolder; + this.rowHolder = rowHolder; } @Override @@ -96,7 +94,7 @@ private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensi return DimensionSelectorUtils.constantSelector(null, extractionFn); } else { final DimensionIndexer indexer = dimensionDesc.getIndexer(); - return indexer.makeDimensionSelector(dimensionSpec, timeAndDimsHolder, dimensionDesc); + return indexer.makeDimensionSelector(dimensionSpec, rowHolder, dimensionDesc); } } @@ -108,41 +106,21 @@ public ColumnValueSelector makeColumnValueSelector(String columnName) } if (columnName.equals(Column.TIME_COLUMN_NAME)) { - class TimeLongColumnSelector implements LongColumnSelector - { - @Override - public long getLong() - { - return timeAndDimsHolder.get().getTimestamp(); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - // nothing to inspect - } - - @Override - public boolean isNull() - { - return false; - } - } - return new TimeLongColumnSelector(); + return rowHolder; } final Integer dimIndex = index.getDimensionIndex(columnName); if (dimIndex != null) { final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(columnName); final DimensionIndexer indexer = dimensionDesc.getIndexer(); - return indexer.makeColumnValueSelector(timeAndDimsHolder, dimensionDesc); + return indexer.makeColumnValueSelector(rowHolder, dimensionDesc); } - return index.makeMetricColumnValueSelector(columnName, timeAndDimsHolder); + return index.makeMetricColumnValueSelector(columnName, rowHolder); } - @Nullable @Override + @Nullable public ColumnCapabilities getColumnCapabilities(String columnName) { if (virtualColumns.exists(columnName)) { diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRow.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRow.java new file mode 100644 index 000000000000..3c0a5fdf8818 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRow.java @@ -0,0 +1,148 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.incremental; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import io.druid.java.util.common.DateTimes; +import io.druid.segment.DimensionIndexer; + +import javax.annotation.Nullable; +import java.lang.reflect.Array; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public final class IncrementalIndexRow +{ + public static final int EMPTY_ROW_INDEX = -1; + + final long timestamp; + final Object[] dims; + private final List dimensionDescsList; + + /** + * rowIndex is not checked in {@link #equals} and {@link #hashCode} on purpose. IncrementalIndexRow acts as a Map key + * and "entry" object (rowIndex is the "value") at the same time. This is done to reduce object indirection and + * improve locality, and avoid boxing of rowIndex as Integer, when stored in JDK collection: + * {@link IncrementalIndex.RollupFactsHolder} needs concurrent collections, that are not present in fastutil. + */ + private int rowIndex; + + IncrementalIndexRow( + long timestamp, + Object[] dims, + List dimensionDescsList + ) + { + this(timestamp, dims, dimensionDescsList, EMPTY_ROW_INDEX); + } + + IncrementalIndexRow( + long timestamp, + Object[] dims, + List dimensionDescsList, + int rowIndex + ) + { + this.timestamp = timestamp; + this.dims = dims; + this.dimensionDescsList = dimensionDescsList; + this.rowIndex = rowIndex; + } + + public long getTimestamp() + { + return timestamp; + } + + public Object[] getDims() + { + return dims; + } + + public int getRowIndex() + { + return rowIndex; + } + + void setRowIndex(int rowIndex) + { + this.rowIndex = rowIndex; + } + + @Override + public String toString() + { + return "IncrementalIndexRow{" + + "timestamp=" + DateTimes.utc(timestamp) + + ", dims=" + Lists.transform( + Arrays.asList(dims), new Function() + { + @Override + public Object apply(@Nullable Object input) + { + if (input == null || Array.getLength(input) == 0) { + return Collections.singletonList("null"); + } + return Collections.singletonList(input); + } + } + ) + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + IncrementalIndexRow that = (IncrementalIndexRow) o; + + if (timestamp != that.timestamp) { + return false; + } + if (dims.length != that.dims.length) { + return false; + } + for (int i = 0; i < dims.length; i++) { + final DimensionIndexer indexer = dimensionDescsList.get(i).getIndexer(); + if (!indexer.checkUnsortedEncodedKeyComponentsEqual(dims[i], that.dims[i])) { + return false; + } + } + return true; + } + + @Override + public int hashCode() + { + int hash = (int) timestamp; + for (int i = 0; i < dims.length; i++) { + final DimensionIndexer indexer = dimensionDescsList.get(i).getIndexer(); + hash = 31 * hash + indexer.getUnsortedEncodedKeyComponentHashCode(dims[i]); + } + return hash; + } +} diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRowHolder.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRowHolder.java new file mode 100644 index 000000000000..54c8468119fa --- /dev/null +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRowHolder.java @@ -0,0 +1,66 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.incremental; + +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.LongColumnSelector; + +/** + * IncrementalIndexRowHolder is a simple {@link #get}/{@link #set} holder of {@link IncrementalIndexRow}. It is used + * to implement various machinery around {@link IncrementalIndex}, e. g. {@link + * IncrementalIndexColumnSelectorFactory}, {@link IncrementalIndexRowIterator}, etc. + * + * By implementing {@link LongColumnSelector}, IncrementalIndexRowHolder plays the role of timestamp column selector, to + * avoid unneeded level of indirection when timestamp column is selected in {@link + * IncrementalIndexColumnSelectorFactory#makeColumnValueSelector(String)}. + */ +public class IncrementalIndexRowHolder implements LongColumnSelector +{ + private IncrementalIndexRow currEntry = null; + + public IncrementalIndexRow get() + { + return currEntry; + } + + public void set(IncrementalIndexRow currEntry) + { + this.currEntry = currEntry; + } + + @Override + public long getLong() + { + return currEntry.getTimestamp(); + } + + @Override + public boolean isNull() + { + // Time column is never null + return false; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } +} diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRowIterator.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRowIterator.java new file mode 100644 index 000000000000..8e8779c17d2e --- /dev/null +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRowIterator.java @@ -0,0 +1,138 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.incremental; + +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.DimensionHandler; +import io.druid.segment.IndexableAdapter; +import io.druid.segment.RowNumCounter; +import io.druid.segment.RowPointer; +import io.druid.segment.TimeAndDimsPointer; +import io.druid.segment.TransformableRowIterator; +import io.druid.segment.VirtualColumns; + +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Implementation of {@link IndexableAdapter#getRows()} for {@link IncrementalIndexAdapter}. + */ +class IncrementalIndexRowIterator implements TransformableRowIterator +{ + private final Iterator timeAndDimsIterator; + + private final IncrementalIndexRowHolder currentRowHolder = new IncrementalIndexRowHolder(); + private final IncrementalIndexRowHolder markedRowHolder = new IncrementalIndexRowHolder(); + /** Initially -1 so that after the first call to {@link #moveToNext()} the row number is 0. */ + private final RowNumCounter currentRowNumCounter = new RowNumCounter(-1); + private final RowPointer currentRowPointer; + private final TimeAndDimsPointer markedRowPointer; + + IncrementalIndexRowIterator(IncrementalIndex incrementalIndex) + { + this.timeAndDimsIterator = incrementalIndex.getFacts().keySet().iterator(); + this.currentRowPointer = makeRowPointer(incrementalIndex, currentRowHolder, currentRowNumCounter); + // markedRowPointer doesn't actually need to be a RowPointer (just a TimeAndDimsPointer), but we create a RowPointer + // in order to reuse the makeRowPointer() method. Passing a dummy RowNumCounter. + this.markedRowPointer = makeRowPointer(incrementalIndex, markedRowHolder, new RowNumCounter()); + } + + private static RowPointer makeRowPointer( + IncrementalIndex incrementalIndex, + IncrementalIndexRowHolder rowHolder, + RowNumCounter rowNumCounter + ) + { + ColumnSelectorFactory columnSelectorFactory = + new IncrementalIndexColumnSelectorFactory(incrementalIndex, VirtualColumns.EMPTY, false, rowHolder); + ColumnValueSelector[] dimensionSelectors = incrementalIndex + .getDimensions() + .stream() + .map(dim -> { + ColumnValueSelector selectorWithUnsortedValues = columnSelectorFactory.makeColumnValueSelector(dim.getName()); + return dim.getIndexer().convertUnsortedValuesToSorted(selectorWithUnsortedValues); + }) + .toArray(ColumnValueSelector[]::new); + List dimensionHandlers = incrementalIndex + .getDimensions() + .stream() + .map(IncrementalIndex.DimensionDesc::getHandler) + .collect(Collectors.toList()); + ColumnValueSelector[] metricSelectors = incrementalIndex + .getMetricNames() + .stream() + .map(columnSelectorFactory::makeColumnValueSelector) + .toArray(ColumnValueSelector[]::new); + + return new RowPointer( + rowHolder, + dimensionSelectors, + dimensionHandlers, + metricSelectors, + incrementalIndex.getMetricNames(), + rowNumCounter + ); + } + + @Override + public boolean moveToNext() + { + if (!timeAndDimsIterator.hasNext()) { + // Do NOT change currentRowHolder, to conform to RowIterator.getPointer() specification. + return false; + } + currentRowHolder.set(timeAndDimsIterator.next()); + currentRowNumCounter.increment(); + return true; + } + + @Override + public RowPointer getPointer() + { + return currentRowPointer; + } + + @Override + public void mark() + { + markedRowHolder.set(currentRowHolder.get()); + } + + @Override + public TimeAndDimsPointer getMarkedPointer() + { + return markedRowPointer; + } + + @Override + public boolean hasTimeAndDimsChangedSinceMark() + { + return !Objects.equals(markedRowHolder.get(), currentRowHolder.get()); + } + + @Override + public void close() + { + // do nothing + } +} diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 2f9b21ed4860..218d6fb65977 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -158,7 +158,15 @@ public ColumnCapabilities getColumnCapabilities(String column) public String getColumnTypeName(String column) { final String metricType = index.getMetricType(column); - return metricType != null ? metricType : getColumnCapabilities(column).getType().toString(); + if (metricType != null) { + return metricType; + } + ColumnCapabilities columnCapabilities = getColumnCapabilities(column); + if (columnCapabilities != null) { + return columnCapabilities.getType().toString(); + } else { + return null; + } } @Override @@ -186,26 +194,17 @@ public Sequence makeCursors( if (!interval.overlaps(dataInterval)) { return Sequences.empty(); } - final Interval actualInterval = interval.overlap(dataInterval); - - Iterable iterable = gran.getIterable(actualInterval); + Iterable intervals = gran.getIterable(actualInterval); if (descending) { - iterable = Lists.reverse(ImmutableList.copyOf(iterable)); + intervals = Lists.reverse(ImmutableList.copyOf(intervals)); } return Sequences - .simple(iterable) + .simple(intervals) .map(i -> new IncrementalIndexCursor(virtualColumns, descending, filter, i, actualInterval, gran)); } - private ValueMatcher makeFilterMatcher(final Filter filter, final Cursor cursor) - { - return filter == null - ? BooleanValueMatcher.of(true) - : filter.makeMatcher(cursor.getColumnSelectorFactory()); - } - @Override public Metadata getMetadata() { @@ -214,12 +213,12 @@ public Metadata getMetadata() private class IncrementalIndexCursor implements Cursor { - private TimeAndDimsHolder currEntry; + private IncrementalIndexRowHolder currEntry; private final ColumnSelectorFactory columnSelectorFactory; private final ValueMatcher filterMatcher; private final int maxRowIndex; - private Iterator baseIter; - private Iterable cursorIterable; + private Iterator baseIter; + private Iterable cursorIterable; private boolean emptyRange; private final DateTime time; private int numAdvanced; @@ -234,9 +233,9 @@ private class IncrementalIndexCursor implements Cursor Granularity gran ) { - currEntry = new TimeAndDimsHolder(); + currEntry = new IncrementalIndexRowHolder(); columnSelectorFactory = new IncrementalIndexColumnSelectorFactory(index, virtualColumns, descending, currEntry); - filterMatcher = makeFilterMatcher(filter, this); + filterMatcher = filter == null ? BooleanValueMatcher.of(true) : filter.makeMatcher(columnSelectorFactory); numAdvanced = -1; maxRowIndex = index.getLastRowIndex(); final long timeStart = Math.max(interval.getStartMillis(), actualInterval.getStartMillis()); @@ -274,7 +273,7 @@ public void advance() while (baseIter.hasNext()) { BaseQuery.checkInterrupted(); - IncrementalIndex.TimeAndDims entry = baseIter.next(); + IncrementalIndexRow entry = baseIter.next(); if (beyondMaxRowIndex(entry.getRowIndex())) { continue; } @@ -302,7 +301,7 @@ public void advanceUninterruptibly() return; } - IncrementalIndex.TimeAndDims entry = baseIter.next(); + IncrementalIndexRow entry = baseIter.next(); if (beyondMaxRowIndex(entry.getRowIndex())) { continue; } @@ -354,7 +353,7 @@ public void reset() boolean foundMatched = false; while (baseIter.hasNext()) { - IncrementalIndex.TimeAndDims entry = baseIter.next(); + IncrementalIndexRow entry = baseIter.next(); if (beyondMaxRowIndex(entry.getRowIndex())) { numAdvanced++; continue; diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java index 23df282a786b..216a4cd706b8 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -144,7 +144,7 @@ protected AddToFactsResult addToFacts( boolean reportParseExceptions, InputRow row, AtomicInteger numEntries, - TimeAndDims key, + IncrementalIndexRow key, ThreadLocal rowContainer, Supplier rowSupplier, boolean skipMaxRowsInMemoryCheck // ignored, we always want to check this for offheap @@ -156,7 +156,7 @@ protected AddToFactsResult addToFacts( synchronized (this) { final int priorIndex = facts.getPriorIndex(key); - if (TimeAndDims.EMPTY_ROW_INDEX != priorIndex) { + if (IncrementalIndexRow.EMPTY_ROW_INDEX != priorIndex) { final int[] indexAndOffset = indexAndOffsets.get(priorIndex); bufferIndex = indexAndOffset[0]; bufferOffset = indexAndOffset[1]; @@ -200,7 +200,7 @@ protected AddToFactsResult addToFacts( } // Last ditch sanity checks - if (numEntries.get() >= maxRowCount && facts.getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX) { + if (numEntries.get() >= maxRowCount && facts.getPriorIndex(key) == IncrementalIndexRow.EMPTY_ROW_INDEX) { throw new IndexSizeExceededException("Maximum number of rows [%d] reached", maxRowCount); } @@ -210,7 +210,7 @@ protected AddToFactsResult addToFacts( // concurrent readers get hold of it and might ask for newly added row indexAndOffsets.add(new int[]{bufferIndex, bufferOffset}); final int prev = facts.putIfAbsent(key, rowIndex); - if (TimeAndDims.EMPTY_ROW_INDEX == prev) { + if (IncrementalIndexRow.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); } else { throw new ISE("WTF! we are in sychronized block."); diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index a961beb27817..0445d77b529d 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -109,7 +109,7 @@ protected AddToFactsResult addToFacts( boolean reportParseExceptions, InputRow row, AtomicInteger numEntries, - TimeAndDims key, + IncrementalIndexRow key, ThreadLocal rowContainer, Supplier rowSupplier, boolean skipMaxRowsInMemoryCheck @@ -120,7 +120,7 @@ protected AddToFactsResult addToFacts( Aggregator[] aggs; - if (TimeAndDims.EMPTY_ROW_INDEX != priorIndex) { + if (IncrementalIndexRow.EMPTY_ROW_INDEX != priorIndex) { aggs = concurrentGet(priorIndex); parseExceptionMessages = doAggregate(metrics, aggs, rowContainer, row); } else { @@ -133,12 +133,12 @@ protected AddToFactsResult addToFacts( // Last ditch sanity checks if (numEntries.get() >= maxRowCount - && facts.getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX + && facts.getPriorIndex(key) == IncrementalIndexRow.EMPTY_ROW_INDEX && !skipMaxRowsInMemoryCheck) { throw new IndexSizeExceededException("Maximum number of rows [%d] reached", maxRowCount); } final int prev = facts.putIfAbsent(key, rowIndex); - if (TimeAndDims.EMPTY_ROW_INDEX == prev) { + if (IncrementalIndexRow.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); } else { // We lost a race diff --git a/processing/src/main/java/io/druid/segment/incremental/package-info.java b/processing/src/main/java/io/druid/segment/incremental/package-info.java new file mode 100644 index 000000000000..de2b96f491f6 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/incremental/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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. + */ + +@EverythingIsNonnullByDefault +package io.druid.segment.incremental; + +import io.druid.annotations.EverythingIsNonnullByDefault; diff --git a/processing/src/main/java/io/druid/segment/selector/settable/SettableColumnValueSelector.java b/processing/src/main/java/io/druid/segment/selector/settable/SettableColumnValueSelector.java new file mode 100644 index 000000000000..8483bbaa8aca --- /dev/null +++ b/processing/src/main/java/io/druid/segment/selector/settable/SettableColumnValueSelector.java @@ -0,0 +1,37 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.selector.settable; + +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.ColumnValueSelector; + +/** + * SettableColumnValueSelectors are used in {@link io.druid.segment.QueryableIndexIndexableAdapter.RowIteratorImpl}. + */ +public interface SettableColumnValueSelector extends ColumnValueSelector +{ + void setValueFrom(ColumnValueSelector selector); + + @Override + default void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // SettableColumnValueSelectors have nothing to inspect + } +} diff --git a/processing/src/main/java/io/druid/segment/selector/settable/SettableDimensionValueSelector.java b/processing/src/main/java/io/druid/segment/selector/settable/SettableDimensionValueSelector.java new file mode 100644 index 000000000000..ded605f60dac --- /dev/null +++ b/processing/src/main/java/io/druid/segment/selector/settable/SettableDimensionValueSelector.java @@ -0,0 +1,112 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.selector.settable; + +import com.google.common.base.Predicate; +import io.druid.query.filter.ValueMatcher; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.DimensionSelector; +import io.druid.segment.IdLookup; +import io.druid.segment.data.ArrayBasedIndexedInts; +import io.druid.segment.data.IndexedInts; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Settable implementation of {@link DimensionSelector}. + */ +public class SettableDimensionValueSelector implements SettableColumnValueSelector, DimensionSelector +{ + @Nullable + private DimensionSelector keptSelector; + private final ArrayBasedIndexedInts keptRow = new ArrayBasedIndexedInts(); + + @Override + public void setValueFrom(ColumnValueSelector selector) + { + DimensionSelector dimensionSelector = (DimensionSelector) selector; + keptSelector = dimensionSelector; + IndexedInts row = dimensionSelector.getRow(); + int rowSize = row.size(); + keptRow.ensureSize(rowSize); + for (int i = 0; i < rowSize; i++) { + keptRow.setValue(i, row.get(i)); + } + keptRow.setSize(rowSize); + } + + @Override + public IndexedInts getRow() + { + return keptRow; + } + + @Override + public ValueMatcher makeValueMatcher(@Nullable String value) + { + throw new UnsupportedOperationException(); + } + + @Override + public ValueMatcher makeValueMatcher(Predicate predicate) + { + throw new UnsupportedOperationException(); + } + + @Override + public int getValueCardinality() + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public String lookupName(int id) + { + return Objects.requireNonNull(keptSelector).lookupName(id); + } + + @Override + public boolean nameLookupPossibleInAdvance() + { + return Objects.requireNonNull(keptSelector).nameLookupPossibleInAdvance(); + } + + @Nullable + @Override + public IdLookup idLookup() + { + return Objects.requireNonNull(keptSelector).idLookup(); + } + + @Nullable + @Override + public Object getObject() + { + return defaultGetObject(); + } + + @Override + public Class classOfObject() + { + return Object.class; + } +} diff --git a/processing/src/main/java/io/druid/segment/selector/settable/SettableDoubleColumnValueSelector.java b/processing/src/main/java/io/druid/segment/selector/settable/SettableDoubleColumnValueSelector.java new file mode 100644 index 000000000000..22d7abdbfa18 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/selector/settable/SettableDoubleColumnValueSelector.java @@ -0,0 +1,54 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.selector.settable; + +import io.druid.common.config.NullHandling; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.DoubleColumnSelector; + +public class SettableDoubleColumnValueSelector implements SettableColumnValueSelector, DoubleColumnSelector +{ + private boolean isNull; + private double value; + + @Override + public void setValueFrom(ColumnValueSelector selector) + { + isNull = selector.isNull(); + if (!isNull) { + value = selector.getDouble(); + } else { + value = 0; + } + } + + @Override + public double getDouble() + { + assert NullHandling.replaceWithDefault() || !isNull; + return value; + } + + @Override + public boolean isNull() + { + return isNull; + } +} diff --git a/processing/src/main/java/io/druid/segment/selector/settable/SettableFloatColumnValueSelector.java b/processing/src/main/java/io/druid/segment/selector/settable/SettableFloatColumnValueSelector.java new file mode 100644 index 000000000000..99b2fbf49495 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/selector/settable/SettableFloatColumnValueSelector.java @@ -0,0 +1,54 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.selector.settable; + +import io.druid.common.config.NullHandling; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.FloatColumnSelector; + +public class SettableFloatColumnValueSelector implements SettableColumnValueSelector, FloatColumnSelector +{ + private boolean isNull; + private float value; + + @Override + public void setValueFrom(ColumnValueSelector selector) + { + isNull = selector.isNull(); + if (!isNull) { + value = selector.getFloat(); + } else { + value = 0; + } + } + + @Override + public float getFloat() + { + assert NullHandling.replaceWithDefault() || !isNull; + return value; + } + + @Override + public boolean isNull() + { + return isNull; + } +} diff --git a/processing/src/main/java/io/druid/segment/selector/settable/SettableLongColumnValueSelector.java b/processing/src/main/java/io/druid/segment/selector/settable/SettableLongColumnValueSelector.java new file mode 100644 index 000000000000..d596e328c893 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/selector/settable/SettableLongColumnValueSelector.java @@ -0,0 +1,60 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.selector.settable; + +import io.druid.common.config.NullHandling; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.LongColumnSelector; + +public class SettableLongColumnValueSelector implements SettableColumnValueSelector, LongColumnSelector +{ + private boolean isNull; + private long value; + + @Override + public void setValueFrom(ColumnValueSelector selector) + { + isNull = selector.isNull(); + if (!isNull) { + value = selector.getLong(); + } else { + value = 0; + } + } + + /** Optimized method for assigning timestamps, that are known to never be null */ + public void setValue(long value) + { + this.value = value; + } + + @Override + public long getLong() + { + assert NullHandling.replaceWithDefault() || !isNull; + return value; + } + + @Override + public boolean isNull() + { + return isNull; + } +} diff --git a/processing/src/main/java/io/druid/segment/selector/settable/SettableObjectColumnValueSelector.java b/processing/src/main/java/io/druid/segment/selector/settable/SettableObjectColumnValueSelector.java new file mode 100644 index 000000000000..f667caba47ef --- /dev/null +++ b/processing/src/main/java/io/druid/segment/selector/settable/SettableObjectColumnValueSelector.java @@ -0,0 +1,53 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.selector.settable; + +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.ObjectColumnSelector; + +import javax.annotation.Nullable; + +public class SettableObjectColumnValueSelector extends ObjectColumnSelector + implements SettableColumnValueSelector +{ + @Nullable + private T object; + + @SuppressWarnings("unchecked") + @Override + public void setValueFrom(ColumnValueSelector selector) + { + object = (T) selector.getObject(); + } + + @Nullable + @Override + public T getObject() + { + return object; + } + + @SuppressWarnings("unchecked") + @Override + public Class classOfObject() + { + return (Class) Object.class; + } +} diff --git a/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java b/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java index 97798eaeddcf..9135b12c19ff 100644 --- a/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java @@ -22,10 +22,11 @@ import io.druid.guice.annotations.PublicApi; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.ColumnValueSelector; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.ObjectStrategy; +import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; import java.nio.channels.WritableByteChannel; @@ -58,11 +59,10 @@ public void open() throws IOException writer.open(); } - @SuppressWarnings(value = "unchecked") @Override - public void serialize(Object obj) throws IOException + public void serialize(ColumnValueSelector selector) throws IOException { - writer.write(obj); + writer.write(selector.getObject()); } @Override diff --git a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index b59b188b3456..64e6a190cd4c 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -54,7 +54,6 @@ import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; - public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde { private static final int NO_FLAGS = 0; diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java index e467683cbf49..1047c018684e 100644 --- a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java @@ -26,8 +26,8 @@ import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressedColumnarDoublesSuppliers; import io.druid.segment.data.ColumnarDoubles; +import io.druid.segment.data.CompressedColumnarDoublesSuppliers; import java.nio.ByteBuffer; import java.nio.ByteOrder; diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java index 046fad51b848..6c91590b6c2d 100644 --- a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java @@ -25,16 +25,12 @@ import io.druid.segment.column.GenericColumn; import io.druid.segment.data.ColumnarDoubles; - public class DoubleGenericColumnSupplier implements Supplier { private final Supplier column; private final ImmutableBitmap nullValueBitmap; - public DoubleGenericColumnSupplier( - Supplier column, - ImmutableBitmap nullValueBitmap - ) + DoubleGenericColumnSupplier(Supplier column, ImmutableBitmap nullValueBitmap) { this.column = column; this.nullValueBitmap = nullValueBitmap; diff --git a/processing/src/main/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializer.java b/processing/src/main/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializer.java index 6486f7eeaa06..a0bf176a3688 100644 --- a/processing/src/main/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializer.java @@ -22,10 +22,11 @@ import io.druid.guice.annotations.PublicApi; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.ColumnValueSelector; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.ObjectStrategy; +import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; import java.nio.channels.WritableByteChannel; @@ -93,11 +94,10 @@ public void open() throws IOException writer.open(); } - @SuppressWarnings(value = "unchecked") @Override - public void serialize(Object obj) throws IOException + public void serialize(ColumnValueSelector selector) throws IOException { - writer.write(obj); + writer.write(selector.getObject()); } @Override diff --git a/processing/src/main/java/io/druid/segment/serde/package-info.java b/processing/src/main/java/io/druid/segment/serde/package-info.java new file mode 100644 index 000000000000..e66436940613 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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. + */ + +@EverythingIsNonnullByDefault +package io.druid.segment.serde; + +import io.druid.annotations.EverythingIsNonnullByDefault; diff --git a/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java b/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java index 3969bfcb99c5..85aa6beb7555 100644 --- a/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java +++ b/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java @@ -25,8 +25,8 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.DimensionSelectorUtils; import io.druid.segment.IdLookup; -import io.druid.segment.data.ArrayBasedIndexedInts; import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.ArrayBasedIndexedInts; import javax.annotation.Nullable; diff --git a/processing/src/test/java/io/druid/segment/DebugRow.java b/processing/src/test/java/io/druid/segment/DebugRow.java new file mode 100644 index 000000000000..f481788a517e --- /dev/null +++ b/processing/src/test/java/io/druid/segment/DebugRow.java @@ -0,0 +1,48 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class DebugRow +{ + final Map dimensions; + final Map metrics; + + public DebugRow(Map dimensions, Map metrics) + { + this.dimensions = dimensions; + this.metrics = metrics; + } + + public List dimensionValues() + { + // Couldn't use ImmutableList because needs to contain nulls + return new ArrayList<>(dimensions.values()); + } + + public List metricValues() + { + // Couldn't use ImmutableList because needs to contain nulls + return new ArrayList<>(metrics.values()); + } +} diff --git a/processing/src/test/java/io/druid/segment/IndexIOTest.java b/processing/src/test/java/io/druid/segment/IndexIOTest.java index 4fefaea23555..b911daf680a5 100644 --- a/processing/src/test/java/io/druid/segment/IndexIOTest.java +++ b/processing/src/test/java/io/druid/segment/IndexIOTest.java @@ -83,7 +83,7 @@ private static List filterByBitset(List list, BitSet bitSet) return outList; } - @Parameterized.Parameters + @Parameterized.Parameters(name = "{0}, {1}") public static Iterable constructionFeeder() { diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java index 6f58c8b1376d..bf6feacc6cc7 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java @@ -56,7 +56,6 @@ import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; import io.druid.segment.incremental.IncrementalIndexSchema; -import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import it.unimi.dsi.fastutil.ints.IntIterator; import org.joda.time.Interval; @@ -166,14 +165,7 @@ public void testPersist() throws Exception final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist, - tempDir, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist, tempDir, indexSpec, null)) ); Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); @@ -201,27 +193,20 @@ public void testPersistWithDifferentDims() throws Exception new MapBasedInputRow( 1, Arrays.asList("dim1", "dim2"), - ImmutableMap.of("dim1", "1", "dim2", "2") + ImmutableMap.of("dim1", "1", "dim2", "2") ) ); toPersist.add( new MapBasedInputRow( 1, - Arrays.asList("dim1"), - ImmutableMap.of("dim1", "3") + Collections.singletonList("dim1"), + ImmutableMap.of("dim1", "3") ) ); final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist, - tempDir, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist, tempDir, indexSpec, null)) ); Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); @@ -230,18 +215,18 @@ public void testPersistWithDifferentDims() throws Exception assertDimCompression(index, indexSpec.getDimensionCompression()); final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); - final List boatList = ImmutableList.copyOf(adapter.getRows()); + final List rowList = RowIteratorHelper.toList(adapter.getRows()); - Assert.assertEquals(2, boatList.size()); - Assert.assertArrayEquals(new int[][]{{0}, {1}}, boatList.get(0).getDims()); - Assert.assertArrayEquals(new int[][]{{1}, {0}}, boatList.get(1).getDims()); + Assert.assertEquals(2, rowList.size()); + Assert.assertEquals(ImmutableList.of("1", "2"), rowList.get(0).dimensionValues()); + Assert.assertEquals(Arrays.asList("3", null), rowList.get(1).dimensionValues()); - checkBitmapIndex(new ArrayList(), adapter.getBitmapIndex("dim1", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim1", "1")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dim1", "3")); + checkBitmapIndex(Collections.emptyList(), adapter.getBitmapIndex("dim1", "")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("dim1", "1")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("dim1", "3")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dim2", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim2", "2")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("dim2", "")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("dim2", "2")); } @Test @@ -252,19 +237,12 @@ public void testPersistWithSegmentMetadata() throws Exception IncrementalIndex toPersist = IncrementalIndexTest.createIndex(null); IncrementalIndexTest.populateIndex(timestamp, toPersist); - Map metadataElems = ImmutableMap.of("key", "value"); + Map metadataElems = ImmutableMap.of("key", "value"); toPersist.getMetadata().putAll(metadataElems); final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist, - tempDir, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist, tempDir, indexSpec, null)) ); Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); @@ -301,7 +279,7 @@ public void testPersistMerge() throws Exception new MapBasedInputRow( timestamp, Arrays.asList("dim1", "dim2"), - ImmutableMap.of("dim1", "1", "dim2", "2") + ImmutableMap.of("dim1", "1", "dim2", "2") ) ); @@ -309,7 +287,7 @@ public void testPersistMerge() throws Exception new MapBasedInputRow( timestamp, Arrays.asList("dim1", "dim2"), - ImmutableMap.of("dim1", "5", "dim2", "6") + ImmutableMap.of("dim1", "5", "dim2", "6") ) ); @@ -318,14 +296,7 @@ public void testPersistMerge() throws Exception final File mergedDir = temporaryFolder.newFolder(); QueryableIndex index1 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist1, - tempDir1, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist1, tempDir1, indexSpec, null)) ); Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); @@ -333,14 +304,7 @@ public void testPersistMerge() throws Exception Assert.assertEquals(3, index1.getColumnNames().size()); QueryableIndex index2 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist2, - tempDir2, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist2, tempDir2, indexSpec, null)) ); Assert.assertEquals(2, index2.getColumn(Column.TIME_COLUMN_NAME).getLength()); @@ -397,7 +361,7 @@ public void testPersistEmptyColumn() throws Exception new MapBasedInputRow( 1L, ImmutableList.of("dim1", "dim2"), - ImmutableMap.of("dim1", ImmutableList.of(), "dim2", "foo") + ImmutableMap.of("dim1", ImmutableList.of(), "dim2", "foo") ) ); @@ -405,29 +369,15 @@ public void testPersistEmptyColumn() throws Exception new MapBasedInputRow( 1L, ImmutableList.of("dim1", "dim2"), - ImmutableMap.of("dim1", ImmutableList.of(), "dim2", "bar") + ImmutableMap.of("dim1", ImmutableList.of(), "dim2", "bar") ) ); final QueryableIndex index1 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist1, - tmpDir1, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist1, tmpDir1, indexSpec, null)) ); final QueryableIndex index2 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist2, - tmpDir2, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist2, tmpDir2, indexSpec, null)) ); final QueryableIndex merged = closer.closeLater( indexIO.loadIndex( @@ -473,14 +423,7 @@ public void testMergeRetainsValues() throws Exception ); QueryableIndex index1 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist1, - tempDir1, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist1, tempDir1, indexSpec, null)) ); @@ -594,14 +537,7 @@ public void testMergeSpecChange() throws Exception ); QueryableIndex index1 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist1, - tempDir1, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist1, tempDir1, indexSpec, null)) ); @@ -689,13 +625,7 @@ public void testConvertSame() throws Exception QueryableIndex converted = closer.closeLater( - indexIO.loadIndex( - indexMerger.convert( - tempDir1, - convertDir, - indexSpec - ) - ) + indexIO.loadIndex(indexMerger.convert(tempDir1, convertDir, indexSpec)) ); Assert.assertEquals(2, converted.getColumn(Column.TIME_COLUMN_NAME).getLength()); @@ -739,14 +669,7 @@ public void testConvertDifferent() throws Exception ); QueryableIndex index1 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist1, - tempDir1, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist1, tempDir1, indexSpec, null)) ); @@ -773,13 +696,7 @@ public void testConvertDifferent() throws Exception ); QueryableIndex converted = closer.closeLater( - indexIO.loadIndex( - indexMerger.convert( - tempDir1, - convertDir, - newSpec - ) - ) + indexIO.loadIndex(indexMerger.convert(tempDir1, convertDir, newSpec)) ); Assert.assertEquals(2, converted.getColumn(Column.TIME_COLUMN_NAME).getLength()); @@ -847,36 +764,15 @@ public void testNonLexicographicDimOrderMerge() throws Exception final File tmpDirMerged = temporaryFolder.newFolder(); QueryableIndex index1 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist1, - tmpDir, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist1, tmpDir, indexSpec, null)) ); QueryableIndex index2 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist2, - tmpDir2, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist2, tmpDir2, indexSpec, null)) ); QueryableIndex index3 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist3, - tmpDir3, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist3, tmpDir3, indexSpec, null)) ); @@ -894,31 +790,34 @@ public void testNonLexicographicDimOrderMerge() throws Exception ); final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); - final List boatList = ImmutableList.copyOf(adapter.getRows()); - - Assert.assertEquals(ImmutableList.of("d3", "d1", "d2"), ImmutableList.copyOf(adapter.getDimensionNames())); - Assert.assertEquals(3, boatList.size()); - Assert.assertArrayEquals(new int[][]{{0}, {0}, {2}}, boatList.get(0).getDims()); - Assert.assertArrayEquals(new Object[]{3L}, boatList.get(0).getMetrics()); - Assert.assertArrayEquals(new int[][]{{1}, {2}, {0}}, boatList.get(1).getDims()); - Assert.assertArrayEquals(new Object[]{3L}, boatList.get(1).getMetrics()); - Assert.assertArrayEquals(new int[][]{{2}, {1}, {1}}, boatList.get(2).getDims()); - Assert.assertArrayEquals(new Object[]{3L}, boatList.get(2).getMetrics()); - - checkBitmapIndex(new ArrayList(), adapter.getBitmapIndex("d3", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d3", "30000")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d3", "40000")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d3", "50000")); - - checkBitmapIndex(new ArrayList(), adapter.getBitmapIndex("d1", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d1", "100")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d1", "200")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d1", "300")); - - checkBitmapIndex(new ArrayList(), adapter.getBitmapIndex("d2", "")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d2", "2000")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d2", "3000")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d2", "4000")); + final List rowList = RowIteratorHelper.toList(adapter.getRows()); + + Assert.assertEquals(Arrays.asList("d3", "d1", "d2"), ImmutableList.copyOf(adapter.getDimensionNames())); + Assert.assertEquals(3, rowList.size()); + + Assert.assertEquals(Arrays.asList("30000", "100", "4000"), rowList.get(0).dimensionValues()); + Assert.assertEquals(Collections.singletonList(3L), rowList.get(0).metricValues()); + + Assert.assertEquals(Arrays.asList("40000", "300", "2000"), rowList.get(1).dimensionValues()); + Assert.assertEquals(Collections.singletonList(3L), rowList.get(1).metricValues()); + + Assert.assertEquals(Arrays.asList("50000", "200", "3000"), rowList.get(2).dimensionValues()); + Assert.assertEquals(Collections.singletonList(3L), rowList.get(2).metricValues()); + + checkBitmapIndex(Collections.emptyList(), adapter.getBitmapIndex("d3", "")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("d3", "30000")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("d3", "40000")); + checkBitmapIndex(Collections.singletonList(2), adapter.getBitmapIndex("d3", "50000")); + + checkBitmapIndex(Collections.emptyList(), adapter.getBitmapIndex("d1", "")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("d1", "100")); + checkBitmapIndex(Collections.singletonList(2), adapter.getBitmapIndex("d1", "200")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("d1", "300")); + + checkBitmapIndex(Collections.emptyList(), adapter.getBitmapIndex("d2", "")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("d2", "2000")); + checkBitmapIndex(Collections.singletonList(2), adapter.getBitmapIndex("d2", "3000")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("d2", "4000")); } @@ -959,36 +858,15 @@ public void testMergeWithDimensionsList() throws Exception final File tmpDirMerged = temporaryFolder.newFolder(); QueryableIndex index1 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist1, - tmpDir, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist1, tmpDir, indexSpec, null)) ); QueryableIndex index2 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist2, - tmpDir2, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist2, tmpDir2, indexSpec, null)) ); QueryableIndex index3 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist3, - tmpDir3, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist3, tmpDir3, indexSpec, null)) ); final QueryableIndex merged = closer.closeLater( @@ -1005,33 +883,39 @@ public void testMergeWithDimensionsList() throws Exception ); final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); - final List boatList = ImmutableList.copyOf(adapter.getRows()); + final List rowList = RowIteratorHelper.toList(adapter.getRows()); Assert.assertEquals(ImmutableList.of("dimA", "dimC"), ImmutableList.copyOf(adapter.getDimensionNames())); - Assert.assertEquals(4, boatList.size()); - Assert.assertArrayEquals(new int[][]{{0}, {1}}, boatList.get(0).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList.get(0).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {2}}, boatList.get(1).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList.get(1).getMetrics()); - Assert.assertArrayEquals(new int[][]{{1}, {0}}, boatList.get(2).getDims()); - Assert.assertArrayEquals(new Object[]{2L}, boatList.get(2).getMetrics()); - Assert.assertArrayEquals(new int[][]{{2}, {0}}, boatList.get(3).getDims()); - Assert.assertArrayEquals(new Object[]{2L}, boatList.get(3).getMetrics()); + Assert.assertEquals(4, rowList.size()); + + Assert.assertEquals(Arrays.asList(null, "1"), rowList.get(0).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList.get(0).metricValues()); + + Assert.assertEquals(Arrays.asList(null, "2"), rowList.get(1).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList.get(1).metricValues()); + + Assert.assertEquals(Arrays.asList("1", null), rowList.get(2).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList.get(2).metricValues()); + + Assert.assertEquals(Arrays.asList("2", null), rowList.get(3).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList.get(3).metricValues()); Assert.assertEquals(useBitmapIndexes, adapter.getCapabilities("dimA").hasBitmapIndexes()); Assert.assertEquals(useBitmapIndexes, adapter.getCapabilities("dimC").hasBitmapIndexes()); if (useBitmapIndexes) { - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dimA", "")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("dimA", "1")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("dimA", "2")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dimA", "")); + checkBitmapIndex(Collections.singletonList(2), adapter.getBitmapIndex("dimA", "1")); + checkBitmapIndex(Collections.singletonList(3), adapter.getBitmapIndex("dimA", "2")); + + checkBitmapIndex(Collections.emptyList(), adapter.getBitmapIndex("dimB", "")); - checkBitmapIndex(Lists.newArrayList(2, 3), adapter.getBitmapIndex("dimC", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dimC", "1")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dimC", "2")); + checkBitmapIndex(Arrays.asList(2, 3), adapter.getBitmapIndex("dimC", "")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("dimC", "1")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("dimC", "2")); } - checkBitmapIndex(new ArrayList<>(), adapter.getBitmapIndex("dimB", "")); + checkBitmapIndex(Collections.emptyList(), adapter.getBitmapIndex("dimB", "")); } @@ -1039,274 +923,82 @@ public void testMergeWithDimensionsList() throws Exception public void testDisjointDimMerge() throws Exception { IncrementalIndex toPersistA = getSingleDimIndex("dimA", Arrays.asList("1", "2")); - IncrementalIndex toPersistB = getSingleDimIndex("dimB", Arrays.asList("1", "2", "3")); + IncrementalIndex toPersistB1 = getSingleDimIndex("dimB", Arrays.asList("1", "2", "3")); IncrementalIndex toPersistB2 = getIndexWithDims(Arrays.asList("dimA", "dimB")); addDimValuesToIndex(toPersistB2, "dimB", Arrays.asList("1", "2", "3")); - final File tmpDirA = temporaryFolder.newFolder(); - final File tmpDirB = temporaryFolder.newFolder(); - final File tmpDirB2 = temporaryFolder.newFolder(); - final File tmpDirMerged = temporaryFolder.newFolder(); - - QueryableIndex indexA = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistA, - tmpDirA, - indexSpec, - null - ) - ) - ); - - QueryableIndex indexB = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistB, - tmpDirB, - indexSpec, - null - ) - ) - ); - - QueryableIndex indexB2 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistB2, - tmpDirB2, - indexSpec, - null - ) - ) - ); - - final QueryableIndex merged = closer.closeLater( - indexIO.loadIndex( - indexMerger.mergeQueryableIndex( - Arrays.asList(indexA, indexB), - true, - new AggregatorFactory[]{new CountAggregatorFactory("count")}, - tmpDirMerged, - indexSpec, - null - ) - ) - ); - - final QueryableIndex merged2 = closer.closeLater( - indexIO.loadIndex( - indexMerger.mergeQueryableIndex( - Arrays.asList(indexA, indexB2), - true, - new AggregatorFactory[]{new CountAggregatorFactory("count")}, - tmpDirMerged, - indexSpec, - null - ) - ) - ); - - final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); - final List boatList = ImmutableList.copyOf(adapter.getRows()); - - final QueryableIndexIndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(merged2); - final List boatList2 = ImmutableList.copyOf(adapter2.getRows()); - - Assert.assertEquals(ImmutableList.of("dimA", "dimB"), ImmutableList.copyOf(adapter.getDimensionNames())); - Assert.assertEquals(5, boatList.size()); - Assert.assertArrayEquals(new int[][]{{0}, {1}}, boatList.get(0).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList.get(0).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {2}}, boatList.get(1).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList.get(1).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {3}}, boatList.get(2).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList.get(2).getMetrics()); - Assert.assertArrayEquals(new int[][]{{1}, {0}}, boatList.get(3).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList.get(3).getMetrics()); - Assert.assertArrayEquals(new int[][]{{2}, {0}}, boatList.get(4).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList.get(4).getMetrics()); - - checkBitmapIndex(Lists.newArrayList(0, 1, 2), adapter.getBitmapIndex("dimA", "")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("dimA", "1")); - checkBitmapIndex(Lists.newArrayList(4), adapter.getBitmapIndex("dimA", "2")); - - checkBitmapIndex(Lists.newArrayList(3, 4), adapter.getBitmapIndex("dimB", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dimB", "1")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dimB", "2")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("dimB", "3")); - - Assert.assertEquals(ImmutableList.of("dimA", "dimB"), ImmutableList.copyOf(adapter2.getDimensionNames())); - Assert.assertEquals(5, boatList2.size()); - Assert.assertArrayEquals(new int[][]{{0}, {1}}, boatList2.get(0).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(0).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {2}}, boatList2.get(1).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(1).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {3}}, boatList2.get(2).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(2).getMetrics()); - Assert.assertArrayEquals(new int[][]{{1}, {0}}, boatList2.get(3).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(3).getMetrics()); - Assert.assertArrayEquals(new int[][]{{2}, {0}}, boatList2.get(4).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(4).getMetrics()); - - // dimA always has bitmap indexes, since it has them in indexA (it comes in through discovery). - Assert.assertTrue(adapter2.getCapabilities("dimA").hasBitmapIndexes()); - checkBitmapIndex(Lists.newArrayList(0, 1, 2), adapter2.getBitmapIndex("dimA", "")); - checkBitmapIndex(Lists.newArrayList(3), adapter2.getBitmapIndex("dimA", "1")); - checkBitmapIndex(Lists.newArrayList(4), adapter2.getBitmapIndex("dimA", "2")); - - // dimB may or may not have bitmap indexes, since it comes in through explicit definition in indexB2. - Assert.assertEquals(useBitmapIndexes, adapter2.getCapabilities("dimB").hasBitmapIndexes()); - if (useBitmapIndexes) { - checkBitmapIndex(Lists.newArrayList(3, 4), adapter2.getBitmapIndex("dimB", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter2.getBitmapIndex("dimB", "1")); - checkBitmapIndex(Lists.newArrayList(1), adapter2.getBitmapIndex("dimB", "2")); - checkBitmapIndex(Lists.newArrayList(2), adapter2.getBitmapIndex("dimB", "3")); - } - } - - @Test - public void testJointDimMerge() throws Exception - { - // (d1, d2, d3) from only one index, and their dim values are ('empty', 'has null', 'no null') - // (d4, d5, d6, d7, d8, d9) are from both indexes - // d4: 'empty' join 'empty' - // d5: 'empty' join 'has null' - // d6: 'empty' join 'no null' - // d7: 'has null' join 'has null' - // d8: 'has null' join 'no null' - // d9: 'no null' join 'no null' + for (IncrementalIndex toPersistB : Arrays.asList(toPersistB1, toPersistB2)) { - IncrementalIndex toPersistA = new IncrementalIndex.Builder() - .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) - .setMaxRowCount(1000) - .buildOnheap(); + final File tmpDirA = temporaryFolder.newFolder(); + final File tmpDirB = temporaryFolder.newFolder(); + final File tmpDirMerged = temporaryFolder.newFolder(); - toPersistA.add( - new MapBasedInputRow( - 1, - Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( - "d1", "", "d2", "", "d3", "310", "d7", "", "d9", "910" - ) - ) - ); - toPersistA.add( - new MapBasedInputRow( - 2, - Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( - "d2", "210", "d3", "311", "d7", "710", "d8", "810", "d9", "911" - ) - ) - ); - - IncrementalIndex toPersistB = new IncrementalIndex.Builder() - .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) - .setMaxRowCount(1000) - .buildOnheap(); - - toPersistB.add( - new MapBasedInputRow( - 3, - Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( - "d5", "520", "d6", "620", "d7", "720", "d8", "820", "d9", "920" - ) - ) - ); - toPersistB.add( - new MapBasedInputRow( - 4, - Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( - "d5", "", "d6", "621", "d7", "", "d8", "821", "d9", "921" - ) - ) - ); - final File tmpDirA = temporaryFolder.newFolder(); - final File tmpDirB = temporaryFolder.newFolder(); - final File tmpDirMerged = temporaryFolder.newFolder(); + QueryableIndex indexA = closer.closeLater( + indexIO.loadIndex(indexMerger.persist(toPersistA, tmpDirA, indexSpec, null)) + ); - QueryableIndex indexA = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistA, - tmpDirA, - indexSpec, - null - ) - ) - ); + QueryableIndex indexB = closer.closeLater( + indexIO.loadIndex(indexMerger.persist(toPersistB, tmpDirB, indexSpec, null)) + ); - QueryableIndex indexB = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistB, - tmpDirB, - indexSpec, - null - ) - ) - ); + final QueryableIndex merged = closer.closeLater( + indexIO.loadIndex( + indexMerger.mergeQueryableIndex( + Arrays.asList(indexA, indexB), + true, + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + tmpDirMerged, + indexSpec, + null + ) + ) + ); - final QueryableIndex merged = closer.closeLater( - indexIO.loadIndex( - indexMerger.mergeQueryableIndex( - Arrays.asList(indexA, indexB), - true, - new AggregatorFactory[]{new CountAggregatorFactory("count")}, - tmpDirMerged, - indexSpec, - null - ) - ) - ); + final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); + final List rowList = RowIteratorHelper.toList(adapter.getRows()); - final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); - final List boatList = ImmutableList.copyOf(adapter.getRows()); + Assert.assertEquals(ImmutableList.of("dimA", "dimB"), ImmutableList.copyOf(adapter.getDimensionNames())); + Assert.assertEquals(5, rowList.size()); - Assert.assertEquals( - ImmutableList.of("d2", "d3", "d5", "d6", "d7", "d8", "d9"), - ImmutableList.copyOf(adapter.getDimensionNames()) - ); - Assert.assertEquals(4, boatList.size()); - Assert.assertArrayEquals(new int[][]{{0}, {1}, {0}, {0}, {0}, {0}, {0}}, boatList.get(0).getDims()); - Assert.assertArrayEquals(new int[][]{{1}, {2}, {0}, {0}, {1}, {1}, {1}}, boatList.get(1).getDims()); - Assert.assertArrayEquals(new int[][]{{0}, {0}, {1}, {1}, {2}, {2}, {2}}, boatList.get(2).getDims()); - Assert.assertArrayEquals(new int[][]{{0}, {0}, {0}, {2}, {0}, {3}, {3}}, boatList.get(3).getDims()); + Assert.assertEquals(Arrays.asList(null, "1"), rowList.get(0).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList.get(0).metricValues()); - checkBitmapIndex(Lists.newArrayList(0, 2, 3), adapter.getBitmapIndex("d2", "")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d2", "210")); + Assert.assertEquals(Arrays.asList(null, "2"), rowList.get(1).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList.get(1).metricValues()); - checkBitmapIndex(Lists.newArrayList(2, 3), adapter.getBitmapIndex("d3", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d3", "310")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d3", "311")); + Assert.assertEquals(Arrays.asList(null, "3"), rowList.get(2).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList.get(2).metricValues()); - checkBitmapIndex(Lists.newArrayList(0, 1, 3), adapter.getBitmapIndex("d5", "")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d5", "520")); + Assert.assertEquals(Arrays.asList("1", null), rowList.get(3).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList.get(3).metricValues()); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("d6", "")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d6", "620")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d6", "621")); + Assert.assertEquals(Arrays.asList("2", null), rowList.get(4).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList.get(4).metricValues()); - checkBitmapIndex(Lists.newArrayList(0, 3), adapter.getBitmapIndex("d7", "")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d7", "710")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d7", "720")); + // dimA always has bitmap indexes, since it has them in indexA (it comes in through discovery). + Assert.assertTrue(adapter.getCapabilities("dimA").hasBitmapIndexes()); + checkBitmapIndex(Arrays.asList(0, 1, 2), adapter.getBitmapIndex("dimA", "")); + checkBitmapIndex(Collections.singletonList(3), adapter.getBitmapIndex("dimA", "1")); + checkBitmapIndex(Collections.singletonList(4), adapter.getBitmapIndex("dimA", "2")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d8", "")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d8", "810")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d8", "820")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d8", "821")); - checkBitmapIndex(new ArrayList(), adapter.getBitmapIndex("d9", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d9", "910")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d9", "911")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d9", "920")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d9", "921")); + // dimB may or may not have bitmap indexes, since it comes in through explicit definition in toPersistB2. + //noinspection ObjectEquality + if (toPersistB == toPersistB2) { + Assert.assertEquals(useBitmapIndexes, adapter.getCapabilities("dimB").hasBitmapIndexes()); + } + //noinspection ObjectEquality + if (toPersistB != toPersistB2 || useBitmapIndexes) { + checkBitmapIndex(Arrays.asList(3, 4), adapter.getBitmapIndex("dimB", "")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("dimB", "1")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("dimB", "2")); + checkBitmapIndex(Collections.singletonList(2), adapter.getBitmapIndex("dimB", "3")); + } + } } @Test - public void testNoRollupMergeWithoutDuplicateRow() throws Exception + public void testJointDimMerge() throws Exception { // (d1, d2, d3) from only one index, and their dim values are ('empty', 'has null', 'no null') // (d4, d5, d6, d7, d8, d9) are from both indexes @@ -1316,138 +1008,135 @@ public void testNoRollupMergeWithoutDuplicateRow() throws Exception // d7: 'has null' join 'has null' // d8: 'has null' join 'no null' // d9: 'no null' join 'no null' + IncrementalIndexSchema rollupIndexSchema = new IncrementalIndexSchema.Builder() + .withMetrics(new CountAggregatorFactory("count")) + .build(); - IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() + IncrementalIndexSchema noRollupIndexSchema = new IncrementalIndexSchema.Builder() .withMetrics(new CountAggregatorFactory("count")) .withRollup(false) .build(); - IncrementalIndex toPersistA = new IncrementalIndex.Builder() - .setIndexSchema(indexSchema) - .setMaxRowCount(1000) - .buildOnheap(); - toPersistA.add( - new MapBasedInputRow( - 1, - Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( - "d1", "", "d2", "", "d3", "310", "d7", "", "d9", "910" - ) - ) - ); - toPersistA.add( - new MapBasedInputRow( - 2, - Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( - "d2", "210", "d3", "311", "d7", "710", "d8", "810", "d9", "911" - ) - ) - ); + for (IncrementalIndexSchema indexSchema : Arrays.asList(rollupIndexSchema, noRollupIndexSchema)) { - IncrementalIndex toPersistB = new IncrementalIndex.Builder() - .setIndexSchema(indexSchema) - .setMaxRowCount(1000) - .buildOnheap(); + IncrementalIndex toPersistA = new IncrementalIndex.Builder() + .setIndexSchema(indexSchema) + .setMaxRowCount(1000) + .buildOnheap(); - toPersistB.add( - new MapBasedInputRow( - 3, - Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( - "d5", "520", "d6", "620", "d7", "720", "d8", "820", "d9", "920" - ) - ) - ); - toPersistB.add( - new MapBasedInputRow( - 4, - Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( - "d5", "", "d6", "621", "d7", "", "d8", "821", "d9", "921" - ) - ) - ); - final File tmpDirA = temporaryFolder.newFolder(); - final File tmpDirB = temporaryFolder.newFolder(); - final File tmpDirMerged = temporaryFolder.newFolder(); + toPersistA.add( + new MapBasedInputRow( + 1, + Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of("d1", "", "d2", "", "d3", "310", "d7", "", "d9", "910") + ) + ); + toPersistA.add( + new MapBasedInputRow( + 2, + Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of("d2", "210", "d3", "311", "d7", "710", "d8", "810", "d9", "911") + ) + ); - QueryableIndex indexA = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistA, - tmpDirA, - indexSpec, - null - ) - ) - ); + IncrementalIndex toPersistB = new IncrementalIndex.Builder() + .setIndexSchema(indexSchema) + .setMaxRowCount(1000) + .buildOnheap(); - QueryableIndex indexB = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistB, - tmpDirB, - indexSpec, - null - ) - ) - ); + toPersistB.add( + new MapBasedInputRow( + 3, + Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of("d5", "520", "d6", "620", "d7", "720", "d8", "820", "d9", "920") + ) + ); + toPersistB.add( + new MapBasedInputRow( + 4, + Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of("d5", "", "d6", "621", "d7", "", "d8", "821", "d9", "921") + ) + ); + final File tmpDirA = temporaryFolder.newFolder(); + final File tmpDirB = temporaryFolder.newFolder(); + final File tmpDirMerged = temporaryFolder.newFolder(); - final QueryableIndex merged = closer.closeLater( - indexIO.loadIndex( - indexMerger.mergeQueryableIndex( - Arrays.asList(indexA, indexB), - true, - new AggregatorFactory[]{new CountAggregatorFactory("count")}, - tmpDirMerged, - indexSpec, - null - ) - ) - ); + QueryableIndex indexA = closer.closeLater( + indexIO.loadIndex(indexMerger.persist(toPersistA, tmpDirA, indexSpec, null)) + ); - final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); - final List boatList = ImmutableList.copyOf(adapter.getRows()); + QueryableIndex indexB = closer.closeLater( + indexIO.loadIndex(indexMerger.persist(toPersistB, tmpDirB, indexSpec, null)) + ); - Assert.assertEquals( - ImmutableList.of("d2", "d3", "d5", "d6", "d7", "d8", "d9"), - ImmutableList.copyOf(adapter.getDimensionNames()) - ); - Assert.assertEquals(4, boatList.size()); - Assert.assertArrayEquals(new int[][]{{0}, {1}, {0}, {0}, {0}, {0}, {0}}, boatList.get(0).getDims()); - Assert.assertArrayEquals(new int[][]{{1}, {2}, {0}, {0}, {1}, {1}, {1}}, boatList.get(1).getDims()); - Assert.assertArrayEquals(new int[][]{{0}, {0}, {1}, {1}, {2}, {2}, {2}}, boatList.get(2).getDims()); - Assert.assertArrayEquals(new int[][]{{0}, {0}, {0}, {2}, {0}, {3}, {3}}, boatList.get(3).getDims()); + final QueryableIndex merged = closer.closeLater( + indexIO.loadIndex( + indexMerger.mergeQueryableIndex( + Arrays.asList(indexA, indexB), + true, + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + tmpDirMerged, + indexSpec, + null + ) + ) + ); - checkBitmapIndex(Lists.newArrayList(0, 2, 3), adapter.getBitmapIndex("d2", "")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d2", "210")); + final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); + final List rowList = RowIteratorHelper.toList(adapter.getRows()); - checkBitmapIndex(Lists.newArrayList(2, 3), adapter.getBitmapIndex("d3", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d3", "310")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d3", "311")); + Assert.assertEquals( + ImmutableList.of("d2", "d3", "d5", "d6", "d7", "d8", "d9"), + ImmutableList.copyOf(adapter.getDimensionNames()) + ); + Assert.assertEquals(4, rowList.size()); + Assert.assertEquals( + Arrays.asList(null, "310", null, null, null, null, "910"), + rowList.get(0).dimensionValues() + ); + Assert.assertEquals( + Arrays.asList("210", "311", null, null, "710", "810", "911"), + rowList.get(1).dimensionValues() + ); + Assert.assertEquals( + Arrays.asList(null, null, "520", "620", "720", "820", "920"), + rowList.get(2).dimensionValues() + ); + Assert.assertEquals( + Arrays.asList(null, null, null, "621", null, "821", "921"), + rowList.get(3).dimensionValues() + ); - checkBitmapIndex(Lists.newArrayList(0, 1, 3), adapter.getBitmapIndex("d5", "")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d5", "520")); + checkBitmapIndex(Arrays.asList(0, 2, 3), adapter.getBitmapIndex("d2", "")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("d2", "210")); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("d6", "")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d6", "620")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d6", "621")); + checkBitmapIndex(Arrays.asList(2, 3), adapter.getBitmapIndex("d3", "")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("d3", "310")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("d3", "311")); - checkBitmapIndex(Lists.newArrayList(0, 3), adapter.getBitmapIndex("d7", "")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d7", "710")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d7", "720")); + checkBitmapIndex(Arrays.asList(0, 1, 3), adapter.getBitmapIndex("d5", "")); + checkBitmapIndex(Collections.singletonList(2), adapter.getBitmapIndex("d5", "520")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d8", "")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d8", "810")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d8", "820")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d8", "821")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("d6", "")); + checkBitmapIndex(Collections.singletonList(2), adapter.getBitmapIndex("d6", "620")); + checkBitmapIndex(Collections.singletonList(3), adapter.getBitmapIndex("d6", "621")); - checkBitmapIndex(new ArrayList(), adapter.getBitmapIndex("d9", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d9", "910")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d9", "911")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d9", "920")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d9", "921")); + checkBitmapIndex(Arrays.asList(0, 3), adapter.getBitmapIndex("d7", "")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("d7", "710")); + checkBitmapIndex(Collections.singletonList(2), adapter.getBitmapIndex("d7", "720")); + + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("d8", "")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("d8", "810")); + checkBitmapIndex(Collections.singletonList(2), adapter.getBitmapIndex("d8", "820")); + checkBitmapIndex(Collections.singletonList(3), adapter.getBitmapIndex("d8", "821")); + + checkBitmapIndex(Collections.emptyList(), adapter.getBitmapIndex("d9", "")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("d9", "910")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("d9", "911")); + checkBitmapIndex(Collections.singletonList(2), adapter.getBitmapIndex("d9", "920")); + checkBitmapIndex(Collections.singletonList(3), adapter.getBitmapIndex("d9", "921")); + } } @Test @@ -1474,7 +1163,7 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception new MapBasedInputRow( 1, Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( + ImmutableMap.of( "d1", "", "d2", "", "d3", "310", "d7", "", "d9", "910" ) ) @@ -1483,7 +1172,7 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception new MapBasedInputRow( 1, Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( + ImmutableMap.of( "d1", "", "d2", "", "d3", "310", "d7", "", "d9", "910" ) ) @@ -1498,7 +1187,7 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception new MapBasedInputRow( 1, Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( + ImmutableMap.of( "d1", "", "d2", "", "d3", "310", "d7", "", "d9", "910" ) ) @@ -1507,7 +1196,7 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception new MapBasedInputRow( 4, Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"), - ImmutableMap.of( + ImmutableMap.of( "d5", "", "d6", "621", "d7", "", "d8", "821", "d9", "921" ) ) @@ -1517,25 +1206,11 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception final File tmpDirMerged = temporaryFolder.newFolder(); QueryableIndex indexA = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistA, - tmpDirA, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersistA, tmpDirA, indexSpec, null)) ); QueryableIndex indexB = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistB, - tmpDirB, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersistB, tmpDirB, indexSpec, null)) ); final QueryableIndex merged = closer.closeLater( @@ -1552,33 +1227,33 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception ); final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); - final List boatList = ImmutableList.copyOf(adapter.getRows()); + final List rowList = RowIteratorHelper.toList(adapter.getRows()); Assert.assertEquals( ImmutableList.of("d3", "d6", "d8", "d9"), ImmutableList.copyOf(adapter.getDimensionNames()) ); - Assert.assertEquals(4, boatList.size()); - Assert.assertArrayEquals(new int[][]{{1}, {0}, {0}, {0}}, boatList.get(0).getDims()); - Assert.assertArrayEquals(new int[][]{{1}, {0}, {0}, {0}}, boatList.get(1).getDims()); - Assert.assertArrayEquals(new int[][]{{1}, {0}, {0}, {0}}, boatList.get(2).getDims()); - Assert.assertArrayEquals(new int[][]{{0}, {1}, {1}, {1}}, boatList.get(3).getDims()); + Assert.assertEquals(4, rowList.size()); + Assert.assertEquals(Arrays.asList("310", null, null, "910"), rowList.get(0).dimensionValues()); + Assert.assertEquals(Arrays.asList("310", null, null, "910"), rowList.get(1).dimensionValues()); + Assert.assertEquals(Arrays.asList("310", null, null, "910"), rowList.get(2).dimensionValues()); + Assert.assertEquals(Arrays.asList(null, "621", "821", "921"), rowList.get(3).dimensionValues()); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d3", "")); - checkBitmapIndex(Lists.newArrayList(0, 1, 2), adapter.getBitmapIndex("d3", "310")); + checkBitmapIndex(Collections.singletonList(3), adapter.getBitmapIndex("d3", "")); + checkBitmapIndex(Arrays.asList(0, 1, 2), adapter.getBitmapIndex("d3", "310")); - checkBitmapIndex(Lists.newArrayList(0, 1, 2), adapter.getBitmapIndex("d6", "")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d6", "621")); + checkBitmapIndex(Arrays.asList(0, 1, 2), adapter.getBitmapIndex("d6", "")); + checkBitmapIndex(Collections.singletonList(3), adapter.getBitmapIndex("d6", "621")); - checkBitmapIndex(Lists.newArrayList(0, 1, 2), adapter.getBitmapIndex("d8", "")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d8", "821")); + checkBitmapIndex(Arrays.asList(0, 1, 2), adapter.getBitmapIndex("d8", "")); + checkBitmapIndex(Collections.singletonList(3), adapter.getBitmapIndex("d8", "821")); - checkBitmapIndex(new ArrayList(), adapter.getBitmapIndex("d9", "")); - checkBitmapIndex(Lists.newArrayList(0, 1, 2), adapter.getBitmapIndex("d9", "910")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d9", "921")); + checkBitmapIndex(Collections.emptyList(), adapter.getBitmapIndex("d9", "")); + checkBitmapIndex(Arrays.asList(0, 1, 2), adapter.getBitmapIndex("d9", "910")); + checkBitmapIndex(Collections.singletonList(3), adapter.getBitmapIndex("d9", "921")); } - private void checkBitmapIndex(ArrayList expected, BitmapValues real) + private void checkBitmapIndex(List expected, BitmapValues real) { Assert.assertEquals("bitmap size", expected.size(), real.size()); int i = 0; @@ -1606,7 +1281,7 @@ public void testMergeWithSupersetOrdering() throws Exception new MapBasedInputRow( 1, Arrays.asList("dimB", "dimA"), - ImmutableMap.of("dimB", "1", "dimA", "") + ImmutableMap.of("dimB", "1", "dimA", "") ) ); @@ -1614,7 +1289,7 @@ public void testMergeWithSupersetOrdering() throws Exception new MapBasedInputRow( 1, Arrays.asList("dimB", "dimA"), - ImmutableMap.of("dimB", "", "dimA", "1") + ImmutableMap.of("dimB", "", "dimA", "1") ) ); @@ -1631,58 +1306,23 @@ public void testMergeWithSupersetOrdering() throws Exception final File tmpDirMerged2 = temporaryFolder.newFolder(); QueryableIndex indexA = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistA, - tmpDirA, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersistA, tmpDirA, indexSpec, null)) ); QueryableIndex indexB = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistB, - tmpDirB, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersistB, tmpDirB, indexSpec, null)) ); QueryableIndex indexBA = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistBA, - tmpDirBA, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersistBA, tmpDirBA, indexSpec, null)) ); QueryableIndex indexBA2 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistBA2, - tmpDirBA2, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersistBA2, tmpDirBA2, indexSpec, null)) ); QueryableIndex indexC = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersistC, - tmpDirC, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersistC, tmpDirC, indexSpec, null)) ); final QueryableIndex merged = closer.closeLater( @@ -1712,82 +1352,89 @@ public void testMergeWithSupersetOrdering() throws Exception ); final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); - final List boatList = ImmutableList.copyOf(adapter.getRows()); + final List rowList = RowIteratorHelper.toList(adapter.getRows()); final QueryableIndexIndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(merged2); - final List boatList2 = ImmutableList.copyOf(adapter2.getRows()); + final List rowList2 = RowIteratorHelper.toList(adapter2.getRows()); Assert.assertEquals(ImmutableList.of("dimB", "dimA"), ImmutableList.copyOf(adapter.getDimensionNames())); - Assert.assertEquals(5, boatList.size()); - Assert.assertArrayEquals(new int[][]{{0}, {1}}, boatList.get(0).getDims()); - Assert.assertArrayEquals(new Object[]{3L}, boatList.get(0).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {2}}, boatList.get(1).getDims()); - Assert.assertArrayEquals(new Object[]{2L}, boatList.get(1).getMetrics()); - Assert.assertArrayEquals(new int[][]{{1}, {0}}, boatList.get(2).getDims()); - Assert.assertArrayEquals(new Object[]{3L}, boatList.get(2).getMetrics()); - Assert.assertArrayEquals(new int[][]{{2}, {0}}, boatList.get(3).getDims()); - Assert.assertArrayEquals(new Object[]{2L}, boatList.get(3).getMetrics()); - Assert.assertArrayEquals(new int[][]{{3}, {0}}, boatList.get(4).getDims()); - Assert.assertArrayEquals(new Object[]{2L}, boatList.get(4).getMetrics()); - - checkBitmapIndex(Lists.newArrayList(2, 3, 4), adapter.getBitmapIndex("dimA", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dimA", "1")); - checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dimA", "2")); - - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dimB", "")); - checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("dimB", "1")); - checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("dimB", "2")); - checkBitmapIndex(Lists.newArrayList(4), adapter.getBitmapIndex("dimB", "3")); + Assert.assertEquals(5, rowList.size()); + + Assert.assertEquals(Arrays.asList(null, "1"), rowList.get(0).dimensionValues()); + Assert.assertEquals(Collections.singletonList(3L), rowList.get(0).metricValues()); + + Assert.assertEquals(Arrays.asList(null, "2"), rowList.get(1).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList.get(1).metricValues()); + + Assert.assertEquals(Arrays.asList("1", null), rowList.get(2).dimensionValues()); + Assert.assertEquals(Collections.singletonList(3L), rowList.get(2).metricValues()); + + Assert.assertEquals(Arrays.asList("2", null), rowList.get(3).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList.get(3).metricValues()); + + Assert.assertEquals(Arrays.asList("3", null), rowList.get(4).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList.get(4).metricValues()); + + checkBitmapIndex(Arrays.asList(2, 3, 4), adapter.getBitmapIndex("dimA", "")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("dimA", "1")); + checkBitmapIndex(Collections.singletonList(1), adapter.getBitmapIndex("dimA", "2")); + + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dimB", "")); + checkBitmapIndex(Collections.singletonList(2), adapter.getBitmapIndex("dimB", "1")); + checkBitmapIndex(Collections.singletonList(3), adapter.getBitmapIndex("dimB", "2")); + checkBitmapIndex(Collections.singletonList(4), adapter.getBitmapIndex("dimB", "3")); Assert.assertEquals(ImmutableList.of("dimA", "dimB", "dimC"), ImmutableList.copyOf(adapter2.getDimensionNames())); - Assert.assertEquals(12, boatList2.size()); - Assert.assertArrayEquals(new int[][]{{0}, {0}, {1}}, boatList2.get(0).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(0).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {0}, {2}}, boatList2.get(1).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(1).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {0}, {3}}, boatList2.get(2).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(2).getMetrics()); - - Assert.assertArrayEquals(new int[][]{{0}, {1}, {0}}, boatList2.get(3).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(3).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {2}, {0}}, boatList2.get(4).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(4).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {3}, {0}}, boatList2.get(5).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(5).getMetrics()); - - Assert.assertArrayEquals(new int[][]{{1}, {0}, {0}}, boatList2.get(6).getDims()); - Assert.assertArrayEquals(new Object[]{3L}, boatList2.get(6).getMetrics()); - Assert.assertArrayEquals(new int[][]{{2}, {0}, {0}}, boatList2.get(7).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(7).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {1}, {0}}, boatList2.get(8).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(8).getMetrics()); - - Assert.assertArrayEquals(new int[][]{{0}, {2}, {0}}, boatList2.get(9).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(9).getMetrics()); - Assert.assertArrayEquals(new int[][]{{0}, {3}, {0}}, boatList2.get(10).getDims()); - Assert.assertArrayEquals(new Object[]{1L}, boatList2.get(10).getMetrics()); - Assert.assertArrayEquals(new int[][]{{2}, {0}, {0}}, boatList2.get(11).getDims()); - Assert.assertArrayEquals(new Object[]{2L}, boatList2.get(11).getMetrics()); - - checkBitmapIndex(Lists.newArrayList(0, 1, 2, 3, 4, 5, 8, 9, 10), adapter2.getBitmapIndex("dimA", "")); - checkBitmapIndex(Lists.newArrayList(6), adapter2.getBitmapIndex("dimA", "1")); - checkBitmapIndex(Lists.newArrayList(7, 11), adapter2.getBitmapIndex("dimA", "2")); - - checkBitmapIndex(Lists.newArrayList(0, 1, 2, 6, 7, 11), adapter2.getBitmapIndex("dimB", "")); - checkBitmapIndex(Lists.newArrayList(3, 8), adapter2.getBitmapIndex("dimB", "1")); - checkBitmapIndex(Lists.newArrayList(4, 9), adapter2.getBitmapIndex("dimB", "2")); - checkBitmapIndex(Lists.newArrayList(5, 10), adapter2.getBitmapIndex("dimB", "3")); - - checkBitmapIndex(Lists.newArrayList(3, 4, 5, 6, 7, 8, 9, 10, 11), adapter2.getBitmapIndex("dimC", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter2.getBitmapIndex("dimC", "1")); - checkBitmapIndex(Lists.newArrayList(1), adapter2.getBitmapIndex("dimC", "2")); - checkBitmapIndex(Lists.newArrayList(2), adapter2.getBitmapIndex("dimC", "3")); + Assert.assertEquals(12, rowList2.size()); + Assert.assertEquals(Arrays.asList(null, null, "1"), rowList2.get(0).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList2.get(0).metricValues()); + Assert.assertEquals(Arrays.asList(null, null, "2"), rowList2.get(1).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList2.get(1).metricValues()); + + Assert.assertEquals(Arrays.asList(null, null, "3"), rowList2.get(2).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList2.get(2).metricValues()); + Assert.assertEquals(Arrays.asList(null, "1", null), rowList2.get(3).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList2.get(3).metricValues()); + + Assert.assertEquals(Arrays.asList(null, "2", null), rowList2.get(4).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList2.get(4).metricValues()); + Assert.assertEquals(Arrays.asList(null, "3", null), rowList2.get(5).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList2.get(5).metricValues()); + + Assert.assertEquals(Arrays.asList("1", null, null), rowList2.get(6).dimensionValues()); + Assert.assertEquals(Collections.singletonList(3L), rowList2.get(6).metricValues()); + Assert.assertEquals(Arrays.asList("2", null, null), rowList2.get(7).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList2.get(7).metricValues()); + + Assert.assertEquals(Arrays.asList(null, "1", null), rowList2.get(8).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList2.get(8).metricValues()); + Assert.assertEquals(Arrays.asList(null, "2", null), rowList2.get(9).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList2.get(9).metricValues()); + + Assert.assertEquals(Arrays.asList(null, "3", null), rowList2.get(10).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList2.get(10).metricValues()); + Assert.assertEquals(Arrays.asList("2", null, null), rowList2.get(11).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList2.get(11).metricValues()); + + checkBitmapIndex(Arrays.asList(0, 1, 2, 3, 4, 5, 8, 9, 10), adapter2.getBitmapIndex("dimA", "")); + checkBitmapIndex(Collections.singletonList(6), adapter2.getBitmapIndex("dimA", "1")); + checkBitmapIndex(Arrays.asList(7, 11), adapter2.getBitmapIndex("dimA", "2")); + + checkBitmapIndex(Arrays.asList(0, 1, 2, 6, 7, 11), adapter2.getBitmapIndex("dimB", "")); + checkBitmapIndex(Arrays.asList(3, 8), adapter2.getBitmapIndex("dimB", "1")); + checkBitmapIndex(Arrays.asList(4, 9), adapter2.getBitmapIndex("dimB", "2")); + checkBitmapIndex(Arrays.asList(5, 10), adapter2.getBitmapIndex("dimB", "3")); + + checkBitmapIndex(Arrays.asList(3, 4, 5, 6, 7, 8, 9, 10, 11), adapter2.getBitmapIndex("dimC", "")); + checkBitmapIndex(Collections.singletonList(0), adapter2.getBitmapIndex("dimC", "1")); + checkBitmapIndex(Collections.singletonList(1), adapter2.getBitmapIndex("dimC", "2")); + checkBitmapIndex(Collections.singletonList(2), adapter2.getBitmapIndex("dimC", "3")); } @Test - public void testMismatchedDimensions() throws IOException, IndexSizeExceededException + public void testMismatchedDimensions() throws IOException { IncrementalIndex index1 = IncrementalIndexTest.createIndex(new AggregatorFactory[]{ new LongSumAggregatorFactory("A", "A") @@ -1795,8 +1442,8 @@ public void testMismatchedDimensions() throws IOException, IndexSizeExceededExce index1.add( new MapBasedInputRow( 1L, - Lists.newArrayList("d1", "d2"), - ImmutableMap.of("d1", "a", "d2", "z", "A", 1) + Arrays.asList("d1", "d2"), + ImmutableMap.of("d1", "a", "d2", "z", "A", 1) ) ); closer.closeLater(index1); @@ -1807,14 +1454,14 @@ public void testMismatchedDimensions() throws IOException, IndexSizeExceededExce }); index2.add(new MapBasedInputRow( 1L, - Lists.newArrayList("d1", "d2"), - ImmutableMap.of("d1", "a", "d2", "z", "A", 2, "C", 100) + Arrays.asList("d1", "d2"), + ImmutableMap.of("d1", "a", "d2", "z", "A", 2, "C", 100) )); closer.closeLater(index2); Interval interval = new Interval(DateTimes.EPOCH, DateTimes.nowUtc()); RoaringBitmapFactory factory = new RoaringBitmapFactory(); - ArrayList toMerge = Lists.newArrayList( + List toMerge = Arrays.asList( new IncrementalIndexAdapter(interval, index1, factory), new IncrementalIndexAdapter(interval, index2, factory) ); @@ -1845,7 +1492,7 @@ public void testAddMetrics() throws IOException new MapBasedInputRow( timestamp, Arrays.asList("dim1", "dim2"), - ImmutableMap.of("dim1", "1", "dim2", "2", "A", 5) + ImmutableMap.of("dim1", "1", "dim2", "2", "A", 5) ) ); @@ -1858,14 +1505,14 @@ public void testAddMetrics() throws IOException new MapBasedInputRow( timestamp, Arrays.asList("dim1", "dim2"), - ImmutableMap.of("dim1", "1", "dim2", "2", "A", 5, "C", 6) + ImmutableMap.of("dim1", "1", "dim2", "2", "A", 5, "C", 6) ) ); closer.closeLater(index2); Interval interval = new Interval(DateTimes.EPOCH, DateTimes.nowUtc()); RoaringBitmapFactory factory = new RoaringBitmapFactory(); - ArrayList toMerge = Lists.newArrayList( + List toMerge = Arrays.asList( new IncrementalIndexAdapter(interval, index1, factory), new IncrementalIndexAdapter(interval, index2, factory) @@ -1876,10 +1523,7 @@ public void testAddMetrics() throws IOException File merged = indexMerger.merge( toMerge, true, - new AggregatorFactory[]{ - new LongSumAggregatorFactory("A", "A"), - new LongSumAggregatorFactory("C", "C") - }, + new AggregatorFactory[]{new LongSumAggregatorFactory("A", "A"), new LongSumAggregatorFactory("C", "C")}, tmpDirMerged, indexSpec ); @@ -1901,7 +1545,7 @@ public void testAddMetricsBothSidesNull() throws IOException new MapBasedInputRow( timestamp, Arrays.asList("dim1", "dim2"), - ImmutableMap.of("dim1", "1", "dim2", "2", "A", 5) + ImmutableMap.of("dim1", "1", "dim2", "2", "A", 5) ) ); @@ -1914,7 +1558,7 @@ public void testAddMetricsBothSidesNull() throws IOException new MapBasedInputRow( timestamp, Arrays.asList("dim1", "dim2"), - ImmutableMap.of("dim1", "1", "dim2", "2", "A", 5, "C", 6) + ImmutableMap.of("dim1", "1", "dim2", "2", "A", 5, "C", 6) ) ); closer.closeLater(index2); @@ -1928,18 +1572,17 @@ public void testAddMetricsBothSidesNull() throws IOException new MapBasedInputRow( timestamp, Arrays.asList("dim1", "dim2"), - ImmutableMap.of("dim1", "1", "dim2", "2", "A", 5) + ImmutableMap.of("dim1", "1", "dim2", "2", "A", 5) ) ); Interval interval = new Interval(DateTimes.EPOCH, DateTimes.nowUtc()); RoaringBitmapFactory factory = new RoaringBitmapFactory(); - ArrayList toMerge = Lists.newArrayList( + List toMerge = Arrays.asList( new IncrementalIndexAdapter(interval, index1, factory), new IncrementalIndexAdapter(interval, index2, factory), new IncrementalIndexAdapter(interval, index3, factory) - ); final File tmpDirMerged = temporaryFolder.newFolder(); @@ -1995,7 +1638,7 @@ public void testMismatchedMetrics() throws IOException Interval interval = new Interval(DateTimes.EPOCH, DateTimes.nowUtc()); RoaringBitmapFactory factory = new RoaringBitmapFactory(); - ArrayList toMerge = Lists.newArrayList( + List toMerge = Arrays.asList( new IncrementalIndexAdapter(interval, index1, factory), new IncrementalIndexAdapter(interval, index2, factory), new IncrementalIndexAdapter(interval, index3, factory), @@ -2044,7 +1687,7 @@ public void testMismatchedMetricsVarying() throws IOException Interval interval = new Interval(DateTimes.EPOCH, DateTimes.nowUtc()); RoaringBitmapFactory factory = new RoaringBitmapFactory(); - ArrayList toMerge = Lists.newArrayList( + List toMerge = Collections.singletonList( new IncrementalIndexAdapter(interval, index2, factory) ); @@ -2061,8 +1704,9 @@ public void testMismatchedMetricsVarying() throws IOException tmpDirMerged, indexSpec ); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(indexIO.loadIndex( - merged))); + final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter( + closer.closeLater(indexIO.loadIndex(merged)) + ); Assert.assertEquals(ImmutableSet.of("A", "B", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); } @@ -2077,25 +1721,11 @@ public void testMergeNumericDims() throws Exception final File tmpDirMerged = temporaryFolder.newFolder(); QueryableIndex index1 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist1, - tmpDir, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist1, tmpDir, indexSpec, null)) ); QueryableIndex index2 = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - toPersist2, - tmpDir2, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(toPersist2, tmpDir2, indexSpec, null)) ); final QueryableIndex merged = closer.closeLater( @@ -2112,23 +1742,22 @@ public void testMergeNumericDims() throws Exception ); final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); - Iterable boats = adapter.getRows(); - List boatList = Lists.newArrayList(boats); + final List rowList = RowIteratorHelper.toList(adapter.getRows()); Assert.assertEquals(ImmutableList.of("dimA", "dimB", "dimC"), ImmutableList.copyOf(adapter.getDimensionNames())); - Assert.assertEquals(4, boatList.size()); + Assert.assertEquals(4, rowList.size()); - Assert.assertArrayEquals(new Object[]{0L, 0.0f, new int[]{2}}, boatList.get(0).getDims()); - Assert.assertArrayEquals(new Object[]{2L}, boatList.get(0).getMetrics()); + Assert.assertEquals(Arrays.asList(0L, 0.0f, "Nully Row"), rowList.get(0).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList.get(0).metricValues()); - Assert.assertArrayEquals(new Object[]{72L, 60000.789f, new int[]{3}}, boatList.get(1).getDims()); - Assert.assertArrayEquals(new Object[]{2L}, boatList.get(0).getMetrics()); + Assert.assertEquals(Arrays.asList(72L, 60000.789f, "World"), rowList.get(1).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList.get(0).metricValues()); - Assert.assertArrayEquals(new Object[]{100L, 4000.567f, new int[]{1}}, boatList.get(2).getDims()); - Assert.assertArrayEquals(new Object[]{2L}, boatList.get(1).getMetrics()); + Assert.assertEquals(Arrays.asList(100L, 4000.567f, "Hello"), rowList.get(2).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList.get(1).metricValues()); - Assert.assertArrayEquals(new Object[]{3001L, 1.2345f, new int[]{0}}, boatList.get(3).getDims()); - Assert.assertArrayEquals(new Object[]{2L}, boatList.get(2).getMetrics()); + Assert.assertEquals(Arrays.asList(3001L, 1.2345f, "Foobar"), rowList.get(3).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList.get(2).metricValues()); } private IncrementalIndex getIndexWithNumericDims() throws Exception @@ -2145,7 +1774,7 @@ private IncrementalIndex getIndexWithNumericDims() throws Exception new MapBasedInputRow( 1, Arrays.asList("dimA", "dimB", "dimC"), - ImmutableMap.of("dimA", 100L, "dimB", 4000.567, "dimC", "Hello") + ImmutableMap.of("dimA", 100L, "dimB", 4000.567, "dimC", "Hello") ) ); @@ -2153,7 +1782,7 @@ private IncrementalIndex getIndexWithNumericDims() throws Exception new MapBasedInputRow( 1, Arrays.asList("dimA", "dimB", "dimC"), - ImmutableMap.of("dimA", 72L, "dimB", 60000.789, "dimC", "World") + ImmutableMap.of("dimA", 72L, "dimB", 60000.789, "dimC", "World") ) ); @@ -2161,7 +1790,7 @@ private IncrementalIndex getIndexWithNumericDims() throws Exception new MapBasedInputRow( 1, Arrays.asList("dimA", "dimB", "dimC"), - ImmutableMap.of("dimA", 3001L, "dimB", 1.2345, "dimC", "Foobar") + ImmutableMap.of("dimA", 3001L, "dimB", 1.2345, "dimC", "Foobar") ) ); @@ -2169,7 +1798,7 @@ private IncrementalIndex getIndexWithNumericDims() throws Exception new MapBasedInputRow( 1, Arrays.asList("dimA", "dimB", "dimC"), - ImmutableMap.of("dimC", "Nully Row") + ImmutableMap.of("dimC", "Nully Row") ) ); @@ -2199,26 +1828,19 @@ public void testPersistNullColumnSkipping() throws Exception }); index1.add(new MapBasedInputRow( 1L, - Lists.newArrayList("d1", "d2"), - ImmutableMap.of("d1", "a", "d2", "", "A", 1) + Arrays.asList("d1", "d2"), + ImmutableMap.of("d1", "a", "d2", "", "A", 1) )); index1.add(new MapBasedInputRow( 1L, - Lists.newArrayList("d1", "d2"), - ImmutableMap.of("d1", "b", "d2", "", "A", 1) + Arrays.asList("d1", "d2"), + ImmutableMap.of("d1", "b", "d2", "", "A", 1) )); final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - indexIO.loadIndex( - indexMerger.persist( - index1, - tempDir, - indexSpec, - null - ) - ) + indexIO.loadIndex(indexMerger.persist(index1, tempDir, indexSpec, null)) ); List expectedColumnNames = Arrays.asList("A", "d1"); List actualColumnNames = Lists.newArrayList(index.getColumnNames()); @@ -2246,7 +1868,7 @@ private IncrementalIndex getIndexD3() throws Exception new MapBasedInputRow( 1, Arrays.asList("d3", "d1", "d2"), - ImmutableMap.of("d1", "100", "d2", "4000", "d3", "30000") + ImmutableMap.of("d1", "100", "d2", "4000", "d3", "30000") ) ); @@ -2254,7 +1876,7 @@ private IncrementalIndex getIndexD3() throws Exception new MapBasedInputRow( 1, Arrays.asList("d3", "d1", "d2"), - ImmutableMap.of("d1", "300", "d2", "2000", "d3", "40000") + ImmutableMap.of("d1", "300", "d2", "2000", "d3", "40000") ) ); @@ -2262,7 +1884,7 @@ private IncrementalIndex getIndexD3() throws Exception new MapBasedInputRow( 1, Arrays.asList("d3", "d1", "d2"), - ImmutableMap.of("d1", "200", "d2", "3000", "d3", "50000") + ImmutableMap.of("d1", "200", "d2", "3000", "d3", "50000") ) ); @@ -2283,13 +1905,7 @@ private IncrementalIndex getSingleDimIndex(String dimName, List values) private void addDimValuesToIndex(IncrementalIndex index, String dimName, List values) throws Exception { for (String val : values) { - index.add( - new MapBasedInputRow( - 1, - Arrays.asList(dimName), - ImmutableMap.of(dimName, val) - ) - ); + index.add(new MapBasedInputRow(1, Collections.singletonList(dimName), ImmutableMap.of(dimName, val))); } } @@ -2354,12 +1970,7 @@ public void testCloser() throws Exception final File v9TmpDir = new File(tempDir, "v9-tmp"); try { - indexMerger.persist( - toPersist, - tempDir, - indexSpec, - null - ); + indexMerger.persist(toPersist, tempDir, indexSpec, null); } finally { if (v8TmpDir.exists()) { @@ -2378,7 +1989,7 @@ public void testMultiValueHandling() throws Exception new MapBasedInputRow( 1, Arrays.asList("dim1", "dim2"), - ImmutableMap.of( + ImmutableMap.of( "dim1", Arrays.asList("x", "a", "a", "b"), "dim2", Arrays.asList("a", "x", "b", "x") ) @@ -2386,7 +1997,7 @@ public void testMultiValueHandling() throws Exception new MapBasedInputRow( 1, Arrays.asList("dim1", "dim2"), - ImmutableMap.of( + ImmutableMap.of( "dim1", Arrays.asList("a", "b", "x"), "dim2", Arrays.asList("x", "a", "b") ) @@ -2396,34 +2007,40 @@ public void testMultiValueHandling() throws Exception List schema; QueryableIndex index; QueryableIndexIndexableAdapter adapter; - List boatList; + List rowList; // xaab-axbx + abx-xab --> aabx-abxx + abx-abx --> abx-abx + aabx-abxx schema = makeDimensionSchemas(Arrays.asList("dim1", "dim2"), MultiValueHandling.SORTED_ARRAY); index = persistAndLoad(schema, rows); adapter = new QueryableIndexIndexableAdapter(index); - boatList = ImmutableList.copyOf(adapter.getRows()); + rowList = RowIteratorHelper.toList(adapter.getRows()); Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); Assert.assertEquals(3, index.getColumnNames().size()); - Assert.assertEquals(2, boatList.size()); - Assert.assertArrayEquals(new int[][]{{0, 1, 2}, {0, 1, 2}}, boatList.get(0).getDims()); - Assert.assertArrayEquals(new int[][]{{0, 0, 1, 2}, {0, 1, 2, 2}}, boatList.get(1).getDims()); + Assert.assertEquals(2, rowList.size()); + Assert.assertEquals( + Arrays.asList(Arrays.asList("a", "b", "x"), Arrays.asList("a", "b", "x")), + rowList.get(0).dimensionValues() + ); + Assert.assertEquals( + Arrays.asList(Arrays.asList("a", "a", "b", "x"), Arrays.asList("a", "b", "x", "x")), + rowList.get(1).dimensionValues() + ); Assert.assertEquals(useBitmapIndexes, adapter.getCapabilities("dim1").hasBitmapIndexes()); Assert.assertEquals(useBitmapIndexes, adapter.getCapabilities("dim2").hasBitmapIndexes()); if (useBitmapIndexes) { - checkBitmapIndex(new ArrayList<>(), adapter.getBitmapIndex("dim1", "")); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim1", "a")); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim1", "b")); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim1", "x")); - - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim2", "a")); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim2", "b")); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim2", "x")); + checkBitmapIndex(Collections.emptyList(), adapter.getBitmapIndex("dim1", "")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim1", "a")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim1", "b")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim1", "x")); + + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim2", "a")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim2", "b")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim2", "x")); } // xaab-axbx + abx-xab --> abx-abx + abx-abx --> abx-abx @@ -2435,23 +2052,26 @@ public void testMultiValueHandling() throws Exception Assert.assertEquals(3, index.getColumnNames().size()); adapter = new QueryableIndexIndexableAdapter(index); - boatList = ImmutableList.copyOf(adapter.getRows()); + rowList = RowIteratorHelper.toList(adapter.getRows()); - Assert.assertEquals(1, boatList.size()); - Assert.assertArrayEquals(new int[][]{{0, 1, 2}, {0, 1, 2}}, boatList.get(0).getDims()); + Assert.assertEquals(1, rowList.size()); + Assert.assertEquals( + Arrays.asList(Arrays.asList("a", "b", "x"), Arrays.asList("a", "b", "x")), + rowList.get(0).dimensionValues() + ); Assert.assertEquals(useBitmapIndexes, adapter.getCapabilities("dim1").hasBitmapIndexes()); Assert.assertEquals(useBitmapIndexes, adapter.getCapabilities("dim2").hasBitmapIndexes()); if (useBitmapIndexes) { - checkBitmapIndex(new ArrayList<>(), adapter.getBitmapIndex("dim1", "")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim1", "a")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim1", "b")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim1", "x")); - - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim2", "a")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim2", "b")); - checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim2", "x")); + checkBitmapIndex(Collections.emptyList(), adapter.getBitmapIndex("dim1", "")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("dim1", "a")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("dim1", "b")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("dim1", "x")); + + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("dim2", "a")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("dim2", "b")); + checkBitmapIndex(Collections.singletonList(0), adapter.getBitmapIndex("dim2", "x")); } // xaab-axbx + abx-xab --> abx-xab + xaab-axbx @@ -2463,24 +2083,30 @@ public void testMultiValueHandling() throws Exception Assert.assertEquals(3, index.getColumnNames().size()); adapter = new QueryableIndexIndexableAdapter(index); - boatList = ImmutableList.copyOf(adapter.getRows()); + rowList = RowIteratorHelper.toList(adapter.getRows()); - Assert.assertEquals(2, boatList.size()); - Assert.assertArrayEquals(new int[][]{{0, 1, 2}, {2, 0, 1}}, boatList.get(0).getDims()); - Assert.assertArrayEquals(new int[][]{{2, 0, 0, 1}, {0, 2, 1, 2}}, boatList.get(1).getDims()); + Assert.assertEquals(2, rowList.size()); + Assert.assertEquals( + Arrays.asList(Arrays.asList("a", "b", "x"), Arrays.asList("x", "a", "b")), + rowList.get(0).dimensionValues() + ); + Assert.assertEquals( + Arrays.asList(Arrays.asList("x", "a", "a", "b"), Arrays.asList("a", "x", "b", "x")), + rowList.get(1).dimensionValues() + ); Assert.assertEquals(useBitmapIndexes, adapter.getCapabilities("dim1").hasBitmapIndexes()); Assert.assertEquals(useBitmapIndexes, adapter.getCapabilities("dim2").hasBitmapIndexes()); if (useBitmapIndexes) { - checkBitmapIndex(new ArrayList<>(), adapter.getBitmapIndex("dim1", "")); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim1", "a")); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim1", "b")); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim1", "x")); - - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim2", "a")); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim2", "b")); - checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("dim2", "x")); + checkBitmapIndex(Collections.emptyList(), adapter.getBitmapIndex("dim1", "")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim1", "a")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim1", "b")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim1", "x")); + + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim2", "a")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim2", "b")); + checkBitmapIndex(Arrays.asList(0, 1), adapter.getBitmapIndex("dim2", "x")); } } diff --git a/processing/src/test/java/io/druid/segment/MergingRowIteratorTest.java b/processing/src/test/java/io/druid/segment/MergingRowIteratorTest.java new file mode 100644 index 000000000000..d4f81629290d --- /dev/null +++ b/processing/src/test/java/io/druid/segment/MergingRowIteratorTest.java @@ -0,0 +1,224 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import com.google.common.collect.Iterators; +import com.google.common.collect.UnmodifiableIterator; +import com.google.common.primitives.Longs; +import io.druid.segment.selector.settable.SettableLongColumnValueSelector; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Deque; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class MergingRowIteratorTest +{ + @Test + public void testEmpty() + { + testMerge(); + } + + @Test + public void testOneEmptyIterator() + { + testMerge(Collections.emptyList()); + } + + @Test + public void testMultipleEmptyIterators() + { + testMerge(Collections.emptyList(), Collections.emptyList(), Collections.emptyList()); + } + + @Test + public void testConstantIterator() + { + testMerge(Longs.asList(1, 1, 1)); + } + + @Test + public void testMultipleConstantIterators() + { + testMerge(Longs.asList(2, 2, 2), Longs.asList(3, 3, 3), Longs.asList(1, 1, 1)); + } + + @Test + public void testAllPossible5ElementSequences() + { + List> possibleSequences = new ArrayList<>(); + populateSequences(possibleSequences, new ArrayDeque<>(), 1, 6, 5); + for (int i1 = 0; i1 < possibleSequences.size(); i1++) { + for (int i2 = i1; i2 < possibleSequences.size(); i2++) { + for (int i3 = i2; i3 < possibleSequences.size(); i3++) { + testMerge(possibleSequences.get(i1), possibleSequences.get(i2), possibleSequences.get(i3)); + } + } + } + } + + private static void populateSequences( + List> possibleSequences, + Deque currentSequence, + int current, + int max, + int maxDepth + ) + { + possibleSequences.add(new ArrayList<>(currentSequence)); + if (currentSequence.size() == maxDepth) { + return; + } + for (int i = current; i < max; i++) { + currentSequence.addLast((long) i); + populateSequences(possibleSequences, currentSequence, i, max, maxDepth); + currentSequence.removeLast(); + } + } + + @SafeVarargs + private static void testMerge(List... timestampSequences) + { + String message = Stream.of(timestampSequences).map(List::toString).collect(Collectors.joining(" ")); + int totalLength = Stream.of(timestampSequences).mapToInt(List::size).sum(); + for (int markIteration = 0; markIteration < totalLength; markIteration++) { + testMerge(message, markIteration, timestampSequences); + } + } + + @SafeVarargs + private static void testMerge(String message, int markIteration, List... timestampSequences) + { + MergingRowIterator mergingRowIterator = new MergingRowIterator( + Stream.of(timestampSequences).map(TestRowIterator::new).collect(Collectors.toList()) + ); + UnmodifiableIterator mergedTimestamps = Iterators.mergeSorted( + Stream.of(timestampSequences).map(List::iterator).collect(Collectors.toList()), + Comparator.naturalOrder() + ); + long markedTimestamp = 0; + long currentTimestamp = 0; + int i = 0; + boolean marked = false; + boolean iterated = false; + while (mergedTimestamps.hasNext()) { + currentTimestamp = mergedTimestamps.next(); + Assert.assertTrue(message, mergingRowIterator.moveToNext()); + iterated = true; + Assert.assertEquals(message, currentTimestamp, mergingRowIterator.getPointer().timestampSelector.getLong()); + if (marked) { + Assert.assertEquals( + message, + markedTimestamp != currentTimestamp, + mergingRowIterator.hasTimeAndDimsChangedSinceMark() + ); + } + if (i == markIteration) { + mergingRowIterator.mark(); + markedTimestamp = currentTimestamp; + marked = true; + } + i++; + } + Assert.assertFalse(message, mergingRowIterator.moveToNext()); + if (iterated) { + Assert.assertEquals(message, currentTimestamp, mergingRowIterator.getPointer().timestampSelector.getLong()); + } + } + + private static class TestRowIterator implements TransformableRowIterator + { + private final Iterator timestamps; + private final RowPointer rowPointer; + private final SettableLongColumnValueSelector currentTimestamp = new SettableLongColumnValueSelector(); + private final RowNumCounter rowNumCounter = new RowNumCounter(); + private final SettableLongColumnValueSelector markedTimestamp = new SettableLongColumnValueSelector(); + private final TimeAndDimsPointer markedRowPointer; + + private TestRowIterator(Iterable timestamps) + { + this.timestamps = timestamps.iterator(); + this.rowPointer = new RowPointer( + currentTimestamp, + ColumnValueSelector.EMPTY_ARRAY, + Collections.emptyList(), + ColumnValueSelector.EMPTY_ARRAY, + Collections.emptyList(), + rowNumCounter + ); + this.markedRowPointer = new TimeAndDimsPointer( + markedTimestamp, + ColumnValueSelector.EMPTY_ARRAY, + Collections.emptyList(), + ColumnValueSelector.EMPTY_ARRAY, + Collections.emptyList() + ); + } + + @Override + public void mark() + { + markedTimestamp.setValueFrom(currentTimestamp); + } + + @Override + public boolean hasTimeAndDimsChangedSinceMark() + { + return markedTimestamp.getLong() != currentTimestamp.getLong(); + } + + @Override + public RowPointer getPointer() + { + return rowPointer; + } + + @Override + public TimeAndDimsPointer getMarkedPointer() + { + return markedRowPointer; + } + + @Override + public boolean moveToNext() + { + if (!timestamps.hasNext()) { + return false; + } + currentTimestamp.setValue(timestamps.next()); + rowNumCounter.increment(); + return true; + } + + @Override + public void close() + { + // do nothing + } + } +} diff --git a/processing/src/test/java/io/druid/segment/RowIteratorHelper.java b/processing/src/test/java/io/druid/segment/RowIteratorHelper.java new file mode 100644 index 000000000000..83f8b0450a92 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/RowIteratorHelper.java @@ -0,0 +1,43 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import java.util.ArrayList; +import java.util.List; + +public final class RowIteratorHelper +{ + public static List toList(RowIterator it) + { + try { + List result = new ArrayList<>(); + while (it.moveToNext()) { + RowPointer row = it.getPointer(); + result.add(new DebugRow(row.getDimensionNamesToValuesForDebug(), row.getMetricNamesToValuesForDebug())); + } + return result; + } + finally { + it.close(); + } + } + + private RowIteratorHelper() {} +} diff --git a/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java b/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java index f31c4a75f3eb..914a48d4589f 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.base.Predicate; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -35,7 +34,6 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.logger.Logger; -import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -45,6 +43,7 @@ import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.serde.ComplexMetrics; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.NoneShardSpec; @@ -505,16 +504,9 @@ public Iterable apply(final TimelineObjectHolder chunk) { try { - return new RowboatFilteringIndexAdapter( + return new RowFilteringIndexAdapter( new QueryableIndexIndexableAdapter(indexIO.loadIndex(chunk.getObject())), - new Predicate() - { - @Override - public boolean apply(Rowboat input) - { - return timelineObjectHolder.getInterval().contains(input.getTimestamp()); - } - } + rowPointer -> timelineObjectHolder.getInterval().contains(rowPointer.getTimestamp()) ); } catch (IOException e) { diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java b/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java index a54483e3bdda..352f3b66d6df 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java @@ -1185,9 +1185,9 @@ public void testDifferentMetrics() DateTimes.of("2011-01-12T00:00:00.000Z"), new TimeseriesResultValue( ImmutableMap.builder() - .put("rows", 11L) + .put("rows", 10L) .put("index", 900.0D) - .put("addRowsIndexConstant", 912.0D) + .put("addRowsIndexConstant", 911.0D) .put("uniques", UNIQUES_1) .put("maxIndex", 100.0D) .put("minIndex", 0.0D) diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java index 91f4005c27d3..e2e4a2e4b7be 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java @@ -80,9 +80,9 @@ public static Collection constructorFeeder() final IncrementalIndex incrementalIndex = SchemalessIndexTest.getIncrementalIndex(); final QueryableIndex persistedIncrementalIndex = TestIndex.persistRealtimeAndLoadMMapped(incrementalIndex); final QueryableIndex mergedIncrementalIndex = schemalessIndexTest.getMergedIncrementalIndex(); - argumentArrays.add(new Object[] {new IncrementalIndexSegment(incrementalIndex, null)}); - argumentArrays.add(new Object[] {new QueryableIndexSegment(null, persistedIncrementalIndex)}); - argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mergedIncrementalIndex)}); + argumentArrays.add(new Object[] {new IncrementalIndexSegment(incrementalIndex, null), false}); + argumentArrays.add(new Object[] {new QueryableIndexSegment(null, persistedIncrementalIndex), false}); + argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mergedIncrementalIndex), true}); } return argumentArrays; } @@ -110,13 +110,13 @@ public static Collection constructorFeeder() Arrays.asList(Intervals.of("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z")) ); - private Segment segment; + private final Segment segment; + private final boolean coalesceAbsentAndEmptyDims; - public SchemalessTestSimpleTest( - Segment segment - ) + public SchemalessTestSimpleTest(Segment segment, boolean coalesceAbsentAndEmptyDims) { this.segment = segment; + this.coalesceAbsentAndEmptyDims = coalesceAbsentAndEmptyDims; } @Test @@ -145,9 +145,9 @@ public void testFullOnTimeseries() DateTimes.of("2011-01-12T00:00:00.000Z"), new TimeseriesResultValue( ImmutableMap.builder() - .put("rows", 11L) + .put("rows", coalesceAbsentAndEmptyDims ? 10L : 11L) .put("index", 900.0) - .put("addRowsIndexConstant", 912.0) + .put("addRowsIndexConstant", coalesceAbsentAndEmptyDims ? 911.0 : 912.0) .put("uniques", 2.000977198748901D) .put("maxIndex", 100.0) .put("minIndex", 0.0) diff --git a/processing/src/test/java/io/druid/segment/StringDimensionHandlerTest.java b/processing/src/test/java/io/druid/segment/StringDimensionHandlerTest.java deleted file mode 100644 index 9897499edaae..000000000000 --- a/processing/src/test/java/io/druid/segment/StringDimensionHandlerTest.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment; - -import com.google.common.collect.ImmutableMap; -import io.druid.data.input.MapBasedInputRow; -import io.druid.data.input.impl.DimensionsSpec; -import io.druid.java.util.common.Intervals; -import io.druid.java.util.common.Pair; -import io.druid.query.aggregation.CountAggregatorFactory; -import io.druid.segment.data.CompressionFactory; -import io.druid.segment.data.CompressionStrategy; -import io.druid.segment.data.ConciseBitmapSerdeFactory; -import io.druid.segment.data.Indexed; -import io.druid.segment.incremental.IncrementalIndex; -import io.druid.segment.incremental.IncrementalIndexAdapter; -import io.druid.segment.incremental.IncrementalIndexSchema; -import org.joda.time.Interval; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -public class StringDimensionHandlerTest -{ - private static final Interval TEST_INTERVAL = Intervals.of("2015-01-01/2015-12-31"); - - private static final IndexSpec INDEX_SPEC = new IndexSpec( - new ConciseBitmapSerdeFactory(), - CompressionStrategy.LZ4, - CompressionStrategy.LZ4, - CompressionFactory.LongEncodingStrategy.LONGS - ); - - - private final List dimensions = Arrays.asList("penguins", "predators"); - - private static Pair getAdapters( - List dims, - Map event1, - Map event2 - ) throws Exception - { - IncrementalIndex incrementalIndex1 = new IncrementalIndex.Builder() - .setIndexSchema( - new IncrementalIndexSchema.Builder() - .withMinTimestamp(TEST_INTERVAL.getStartMillis()) - .withDimensionsSpec(new DimensionsSpec(DimensionsSpec.getDefaultSchemas(dims), null, null)) - .withMetrics(new CountAggregatorFactory("count")) - .build() - ) - .setMaxRowCount(1000) - .buildOnheap(); - - IncrementalIndex incrementalIndex2 = new IncrementalIndex.Builder() - .setIndexSchema( - new IncrementalIndexSchema.Builder() - .withMinTimestamp(TEST_INTERVAL.getStartMillis()) - .withDimensionsSpec(new DimensionsSpec(DimensionsSpec.getDefaultSchemas(dims), null, null)) - .withMetrics(new CountAggregatorFactory("count")) - .build() - ) - .setMaxRowCount(1000) - .buildOnheap(); - - incrementalIndex1.add(new MapBasedInputRow(TEST_INTERVAL.getStartMillis(), dims, event1)); - incrementalIndex2.add(new MapBasedInputRow(TEST_INTERVAL.getStartMillis() + 3, dims, event2)); - - IncrementalIndexAdapter adapter1 = new IncrementalIndexAdapter( - TEST_INTERVAL, - incrementalIndex1, - INDEX_SPEC.getBitmapSerdeFactory().getBitmapFactory() - ); - IncrementalIndexAdapter adapter2 = new IncrementalIndexAdapter( - TEST_INTERVAL, - incrementalIndex2, - INDEX_SPEC.getBitmapSerdeFactory().getBitmapFactory() - ); - - return new Pair<>(adapter1, adapter2); - } - - private static void validate(IncrementalIndexAdapter adapter1, IncrementalIndexAdapter adapter2) - { - Map handlers = adapter1.getDimensionHandlers(); - Indexed dimNames1 = adapter1.getDimensionNames(); - Indexed dimNames2 = adapter2.getDimensionNames(); - Iterator iterator1 = adapter1.getRows().iterator(); - Iterator iterator2 = adapter2.getRows().iterator(); - - while (iterator1.hasNext()) { - Rowboat r1 = iterator1.next(); - Rowboat r2 = iterator2.next(); - Object[] dims1 = r1.getDims(); - Object[] dims2 = r2.getDims(); - for (int i = 0; i < dims1.length; i++) { - Object val1 = dims1[i]; - Object val2 = dims2[i]; - String name1 = dimNames1.get(i); - String name2 = dimNames2.get(i); - DimensionHandler handler = handlers.get(name1); - handler.validateSortedEncodedKeyComponents( - val1, - val2, - adapter1.getDimValueLookup(name1), - adapter2.getDimValueLookup(name2) - ); - } - } - } - - @Test - public void testValidateSortedEncodedArrays() throws Exception - { - Map event1 = ImmutableMap.of( - "penguins", Arrays.asList("adelie", "emperor"), - "predators", Arrays.asList("seal") - ); - Map event2 = ImmutableMap.of( - "penguins", Arrays.asList("adelie", "emperor"), - "predators", Arrays.asList("seal") - ); - - Pair adapters = getAdapters(dimensions, event1, event2); - IncrementalIndexAdapter adapter1 = adapters.lhs; - IncrementalIndexAdapter adapter2 = adapters.rhs; - - validate(adapter1, adapter2); - } - - @Rule - public ExpectedException exception = ExpectedException.none(); - - @Test - public void testValidateSortedDifferentEncodedArrays() throws Exception - { - Map event1 = ImmutableMap.of( - "penguins", Arrays.asList("adelie", "emperor"), - "predators", Collections.singletonList("seal") - ); - Map event2 = ImmutableMap.of( - "penguins", Arrays.asList("chinstrap", "gentoo"), - "predators", Collections.singletonList("seal") - ); - - Pair adapters = getAdapters(dimensions, event1, event2); - IncrementalIndexAdapter adapter1 = adapters.lhs; - IncrementalIndexAdapter adapter2 = adapters.rhs; - - exception.expect(SegmentValidationException.class); - validate(adapter1, adapter2); - } -} diff --git a/processing/src/test/java/io/druid/segment/data/CompressedColumnarIntsSerializerTest.java b/processing/src/test/java/io/druid/segment/data/CompressedColumnarIntsSerializerTest.java index f2fef65beb67..586b41c4da9f 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedColumnarIntsSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedColumnarIntsSerializerTest.java @@ -126,7 +126,7 @@ private void checkSerializedSizeAndData(int chunkFactor) throws Exception ); writer.open(); for (int val : vals) { - writer.add(val); + writer.addValue(val); } long writtenLength = writer.getSerializedSize(); final WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); @@ -204,7 +204,7 @@ private void checkV2SerializedSizeAndData(int chunkFactor) throws Exception writer.open(); for (int val : vals) { - writer.add(val); + writer.addValue(val); } final SmooshedWriter channel = smoosher.addWithSmooshedWriter("test", writer.getSerializedSize()); writer.writeTo(channel, smoosher); diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeColumnarIntsSerializerTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeColumnarIntsSerializerTest.java index b6b331804f30..dda006eda06e 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeColumnarIntsSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeColumnarIntsSerializerTest.java @@ -129,7 +129,7 @@ private void checkSerializedSizeAndData(int chunkSize) throws Exception ); writer.open(); for (int val : vals) { - writer.add(val); + writer.addValue(val); } long writtenLength = writer.getSerializedSize(); final WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); @@ -200,7 +200,7 @@ private void checkV2SerializedSizeAndData(int chunkSize) throws Exception ); writer.open(); for (int val : vals) { - writer.add(val); + writer.addValue(val); } final SmooshedWriter channel = smoosher.addWithSmooshedWriter( diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeColumnarMultiIntsSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeColumnarMultiIntsSupplierTest.java index 7feae6d80f6c..f575a2085168 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeColumnarMultiIntsSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeColumnarMultiIntsSupplierTest.java @@ -42,7 +42,7 @@ public class CompressedVSizeColumnarMultiIntsSupplierTest private Closer closer; protected List vals; - protected WritableSupplier indexedSupplier; + protected WritableSupplier columnarMultiIntsSupplier; @Before public void setUpSimple() @@ -55,7 +55,7 @@ public void setUpSimple() new int[]{11, 12, 13, 14, 15, 16, 17, 18, 19, 20} ); - indexedSupplier = CompressedVSizeColumnarMultiIntsSupplier.fromIterable( + columnarMultiIntsSupplier = CompressedVSizeColumnarMultiIntsSupplier.fromIterable( Iterables.transform(vals, input -> VSizeColumnarInts.fromArray(input, 20)), 20, ByteOrder.nativeOrder(), @@ -67,7 +67,7 @@ public void setUpSimple() @After public void teardown() throws IOException { - indexedSupplier = null; + columnarMultiIntsSupplier = null; vals = null; closer.close(); } @@ -75,32 +75,32 @@ public void teardown() throws IOException @Test public void testSanity() { - assertSame(vals, indexedSupplier.get()); + assertSame(vals, columnarMultiIntsSupplier.get()); } @Test public void testSerde() throws IOException { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - indexedSupplier.writeTo(Channels.newChannel(baos), null); + columnarMultiIntsSupplier.writeTo(Channels.newChannel(baos), null); final byte[] bytes = baos.toByteArray(); - Assert.assertEquals(indexedSupplier.getSerializedSize(), bytes.length); - WritableSupplier deserializedIndexed = fromByteBuffer(ByteBuffer.wrap(bytes)); + Assert.assertEquals(columnarMultiIntsSupplier.getSerializedSize(), bytes.length); + WritableSupplier deserializedColumnarMultiInts = fromByteBuffer(ByteBuffer.wrap(bytes)); - assertSame(vals, deserializedIndexed.get()); + assertSame(vals, deserializedColumnarMultiInts.get()); } @Test(expected = IllegalArgumentException.class) public void testGetInvalidElementInRow() { - indexedSupplier.get().get(3).get(15); + columnarMultiIntsSupplier.get().get(3).get(15); } @Test public void testIterators() { - Iterator iterator = indexedSupplier.get().iterator(); + Iterator iterator = columnarMultiIntsSupplier.get().iterator(); int row = 0; while (iterator.hasNext()) { final int[] ints = vals.get(row); @@ -114,12 +114,12 @@ public void testIterators() } } - private void assertSame(List someInts, ColumnarMultiInts indexed) + private void assertSame(List someInts, ColumnarMultiInts columnarMultiInts) { - Assert.assertEquals(someInts.size(), indexed.size()); - for (int i = 0; i < indexed.size(); ++i) { + Assert.assertEquals(someInts.size(), columnarMultiInts.size()); + for (int i = 0; i < columnarMultiInts.size(); ++i) { final int[] ints = someInts.get(i); - final IndexedInts vSizeIndexedInts = indexed.get(i); + final IndexedInts vSizeIndexedInts = columnarMultiInts.get(i); Assert.assertEquals(ints.length, vSizeIndexedInts.size()); for (int j = 0; j < ints.length; j++) { diff --git a/processing/src/test/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializerTest.java b/processing/src/test/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializerTest.java index 773f7eec7cbe..8a5155913068 100644 --- a/processing/src/test/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializerTest.java @@ -134,7 +134,7 @@ private void checkSerializedSizeAndData(int offsetChunkFactor, int valueChunkFac ); writer.open(); for (int[] val : vals) { - writer.add(val); + writer.addValues(new ArrayBasedIndexedInts(val)); } long writtenLength = writer.getSerializedSize(); final WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); @@ -256,10 +256,11 @@ private void checkV2SerializedSizeAndData(int offsetChunkFactor, int valueChunkF compressionStrategy, genericIndexed ); - V3CompressedVSizeColumnarMultiIntsSerializer writer = new V3CompressedVSizeColumnarMultiIntsSerializer(offsetWriter, valueWriter); + V3CompressedVSizeColumnarMultiIntsSerializer writer = + new V3CompressedVSizeColumnarMultiIntsSerializer(offsetWriter, valueWriter); writer.open(); for (int[] val : vals) { - writer.add(val); + writer.addValues(new ArrayBasedIndexedInts(val)); } final SmooshedWriter channel = smoosher.addWithSmooshedWriter("test", writer.getSerializedSize()); @@ -268,10 +269,8 @@ private void checkV2SerializedSizeAndData(int offsetChunkFactor, int valueChunkF smoosher.close(); SmooshedFileMapper mapper = Smoosh.map(tmpDirectory); - V3CompressedVSizeColumnarMultiIntsSupplier supplierFromByteBuffer = V3CompressedVSizeColumnarMultiIntsSupplier.fromByteBuffer( - mapper.mapFile("test"), - byteOrder - ); + V3CompressedVSizeColumnarMultiIntsSupplier supplierFromByteBuffer = + V3CompressedVSizeColumnarMultiIntsSupplier.fromByteBuffer(mapper.mapFile("test"), byteOrder); ColumnarMultiInts columnarMultiInts = supplierFromByteBuffer.get(); assertEquals(columnarMultiInts.size(), vals.size()); for (int i = 0; i < vals.size(); ++i) { diff --git a/processing/src/test/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSupplierTest.java b/processing/src/test/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSupplierTest.java index 236c6eb8957a..403f3650c429 100644 --- a/processing/src/test/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSupplierTest.java @@ -46,7 +46,7 @@ public void setUpSimple() new int[]{11, 12, 13, 14, 15, 16, 17, 18, 19, 20} ); closer = Closer.create(); - indexedSupplier = V3CompressedVSizeColumnarMultiIntsSupplier.fromIterable( + columnarMultiIntsSupplier = V3CompressedVSizeColumnarMultiIntsSupplier.fromIterable( Iterables.transform(vals, (Function) input -> VSizeColumnarInts.fromArray(input, 20)), 2, 20, @@ -60,7 +60,7 @@ public void setUpSimple() @After public void teardown() throws IOException { - indexedSupplier = null; + columnarMultiIntsSupplier = null; closer.close(); vals = null; } diff --git a/processing/src/test/java/io/druid/segment/data/VSizeColumnarIntsSerializerTest.java b/processing/src/test/java/io/druid/segment/data/VSizeColumnarIntsSerializerTest.java index 568a56e96005..8b618b09f648 100644 --- a/processing/src/test/java/io/druid/segment/data/VSizeColumnarIntsSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/VSizeColumnarIntsSerializerTest.java @@ -23,7 +23,6 @@ import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.writeout.WriteOutBytes; -import it.unimi.dsi.fastutil.ints.IntArrayList; import org.apache.commons.io.IOUtils; import org.junit.After; import org.junit.Before; @@ -67,12 +66,10 @@ private void checkSerializedSizeAndData() throws Exception int maxValue = vals.length == 0 ? 0 : Ints.max(vals); VSizeColumnarIntsSerializer writer = new VSizeColumnarIntsSerializer(segmentWriteOutMedium, maxValue); - VSizeColumnarInts intsFromList = VSizeColumnarInts.fromList( - IntArrayList.wrap(vals), maxValue - ); + VSizeColumnarInts intsFromList = VSizeColumnarInts.fromIndexedInts(new ArrayBasedIndexedInts(vals), maxValue); writer.open(); for (int val : vals) { - writer.add(val); + writer.addValue(val); } long writtenLength = writer.getSerializedSize(); WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); diff --git a/processing/src/test/java/io/druid/segment/filter/RowboatTest.java b/processing/src/test/java/io/druid/segment/filter/RowboatTest.java deleted file mode 100644 index 4c80b109bee3..000000000000 --- a/processing/src/test/java/io/druid/segment/filter/RowboatTest.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.filter; - -import io.druid.segment.DimensionHandler; -import io.druid.segment.Rowboat; -import io.druid.segment.StringDimensionHandler; -import org.junit.Assert; -import org.junit.Test; - -/** - * - */ -public class RowboatTest -{ - private static DimensionHandler[] getDefaultHandlers(int size) - { - DimensionHandler[] handlers = new DimensionHandler[size]; - for (int i = 0; i < size; i++) { - handlers[i] = new StringDimensionHandler(String.valueOf(i), null, true); - } - return handlers; - } - - @Test - public void testRowboatCompare() - { - DimensionHandler[] handlers = getDefaultHandlers(3); - Rowboat rb1 = new Rowboat(12345L, new int[][]{new int[]{1}, new int[]{2}}, new Object[]{new Integer(7)}, 5, handlers); - Rowboat rb2 = new Rowboat(12345L, new int[][]{new int[]{1}, new int[]{2}}, new Object[]{new Integer(7)}, 5, handlers); - Assert.assertEquals(0, rb1.compareTo(rb2)); - - Rowboat rb3 = new Rowboat(12345L, new int[][]{new int[]{3}, new int[]{2}}, new Object[]{new Integer(7)}, 5, handlers); - Assert.assertNotEquals(0, rb1.compareTo(rb3)); - } - - @Test - public void testBiggerCompare() - { - DimensionHandler[] handlers = getDefaultHandlers(14); - - Rowboat rb1 = new Rowboat( - 0, - new int[][]{ - new int[]{0}, - new int[]{138}, - new int[]{44}, - new int[]{374}, - new int[]{0}, - new int[]{0}, - new int[]{552}, - new int[]{338}, - new int[]{910}, - new int[]{25570}, - new int[]{9}, - new int[]{0}, - new int[]{0}, - new int[]{0} - }, - new Object[]{1.0, 47.0, "someMetric"}, - 0, - handlers - ); - - Rowboat rb2 = new Rowboat( - 0, - new int[][]{ - new int[]{0}, - new int[]{138}, - new int[]{44}, - new int[]{374}, - new int[]{0}, - new int[]{0}, - new int[]{553}, - new int[]{338}, - new int[]{910}, - new int[]{25580}, - new int[]{9}, - new int[]{0}, - new int[]{0}, - new int[]{0} - }, - new Object[]{1.0, 47.0, "someMetric"}, - 0, - handlers - ); - - Assert.assertNotEquals(0, rb1.compareTo(rb2)); - } - - @Test - public void testToString() - { - DimensionHandler[] handlers = getDefaultHandlers(2); - - Assert.assertEquals( - "Rowboat{timestamp=1970-01-01T00:00:00.000Z, dims=[[1], [2]], metrics=[someMetric], comprisedRows={}}", - new Rowboat(0, new int[][]{new int[]{1}, new int[]{2}}, new Object[]{"someMetric"}, 5, handlers).toString() - ); - } - - @Test - public void testLotsONullString() - { - DimensionHandler[] handlers = getDefaultHandlers(0); - - Assert.assertEquals( - "Rowboat{timestamp=1970-01-01T00:00:00.000Z, dims=null, metrics=null, comprisedRows={}}", - new Rowboat(0, null, null, 5, handlers).toString() - ); - } -} diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java index 0e7a4e91cb3d..cb159498a7ff 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java @@ -21,7 +21,7 @@ import io.druid.segment.IndexSpec; import io.druid.segment.IndexableAdapter; -import io.druid.segment.Rowboat; +import io.druid.segment.RowIterator; import io.druid.segment.data.BitmapValues; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.CompressionStrategy; @@ -74,36 +74,13 @@ public void testGetRowsIterable() throws Exception .getBitmapFactory() ); - Iterable boats = incrementalAdapter.getRows(); - List boatList = new ArrayList<>(); - for (Rowboat boat : boats) { - boatList.add(boat); + RowIterator rows = incrementalAdapter.getRows(); + List rowNums = new ArrayList<>(); + while (rows.moveToNext()) { + rowNums.add(rows.getPointer().getRowNum()); } - Assert.assertEquals(2, boatList.size()); - Assert.assertEquals(0, boatList.get(0).getRowNum()); - Assert.assertEquals(1, boatList.get(1).getRowNum()); - - /* Iterate through the Iterable a few times, check that boat row numbers are correct afterwards */ - boatList = new ArrayList<>(); - for (Rowboat boat : boats) { - boatList.add(boat); - } - boatList = new ArrayList<>(); - for (Rowboat boat : boats) { - boatList.add(boat); - } - boatList = new ArrayList<>(); - for (Rowboat boat : boats) { - boatList.add(boat); - } - boatList = new ArrayList<>(); - for (Rowboat boat : boats) { - boatList.add(boat); - } - - Assert.assertEquals(2, boatList.size()); - Assert.assertEquals(0, boatList.get(0).getRowNum()); - Assert.assertEquals(1, boatList.get(1).getRowNum()); - + Assert.assertEquals(2, rowNums.size()); + Assert.assertEquals(0, (long) rowNums.get(0)); + Assert.assertEquals(1, (long) rowNums.get(1)); } } diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexRowCompTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexRowCompTest.java new file mode 100644 index 000000000000..1071be563f75 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexRowCompTest.java @@ -0,0 +1,85 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.incremental; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import io.druid.data.input.MapBasedInputRow; +import io.druid.query.aggregation.CountAggregatorFactory; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.Map; + +/** + */ +public class IncrementalIndexRowCompTest +{ + @Test + public void testBasic() + { + IncrementalIndex index = new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) + .setMaxRowCount(1000) + .buildOnheap(); + + long time = System.currentTimeMillis(); + IncrementalIndexRow ir1 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "B")).getIncrementalIndexRow(); + IncrementalIndexRow ir2 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "A")).getIncrementalIndexRow(); + IncrementalIndexRow ir3 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A")).getIncrementalIndexRow(); + + IncrementalIndexRow ir4 = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", "B")).getIncrementalIndexRow(); + IncrementalIndexRow ir5 = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", Arrays.asList("A", "B"))).getIncrementalIndexRow(); + IncrementalIndexRow ir6 = index.toIncrementalIndexRow(toMapRow(time + 1)).getIncrementalIndexRow(); + + Comparator comparator = index.dimsComparator(); + + Assert.assertEquals(0, comparator.compare(ir1, ir1)); + Assert.assertEquals(0, comparator.compare(ir2, ir2)); + Assert.assertEquals(0, comparator.compare(ir3, ir3)); + + Assert.assertTrue(comparator.compare(ir1, ir2) > 0); + Assert.assertTrue(comparator.compare(ir2, ir1) < 0); + Assert.assertTrue(comparator.compare(ir2, ir3) > 0); + Assert.assertTrue(comparator.compare(ir3, ir2) < 0); + Assert.assertTrue(comparator.compare(ir1, ir3) > 0); + Assert.assertTrue(comparator.compare(ir3, ir1) < 0); + + Assert.assertTrue(comparator.compare(ir6, ir1) > 0); + Assert.assertTrue(comparator.compare(ir6, ir2) > 0); + Assert.assertTrue(comparator.compare(ir6, ir3) > 0); + + Assert.assertTrue(comparator.compare(ir4, ir6) > 0); + Assert.assertTrue(comparator.compare(ir5, ir6) > 0); + Assert.assertTrue(comparator.compare(ir4, ir5) < 0); + Assert.assertTrue(comparator.compare(ir5, ir4) > 0); + } + + private MapBasedInputRow toMapRow(long time, Object... dimAndVal) + { + Map data = Maps.newHashMap(); + for (int i = 0; i < dimAndVal.length; i += 2) { + data.put((String) dimAndVal[i], dimAndVal[i + 1]); + } + return new MapBasedInputRow(time, Lists.newArrayList(data.keySet()), data); + } +} diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index 53ee9ec92c63..94734aafad59 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -172,7 +172,7 @@ protected AddToFactsResult addToFacts( boolean reportParseExceptions, InputRow row, AtomicInteger numEntries, - TimeAndDims key, + IncrementalIndexRow key, ThreadLocal rowContainer, Supplier rowSupplier, boolean skipMaxRowsInMemoryCheck // ignore for benchmark @@ -202,11 +202,11 @@ protected AddToFactsResult addToFacts( // Last ditch sanity checks - if (numEntries.get() >= maxRowCount && getFacts().getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX) { + if (numEntries.get() >= maxRowCount && getFacts().getPriorIndex(key) == IncrementalIndexRow.EMPTY_ROW_INDEX) { throw new IndexSizeExceededException("Maximum number of rows reached"); } final int prev = getFacts().putIfAbsent(key, rowIndex); - if (TimeAndDims.EMPTY_ROW_INDEX == prev) { + if (IncrementalIndexRow.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); } else { // We lost a race diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java index 4021f1e3025c..adaba6a29d24 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java @@ -84,7 +84,7 @@ public void run() public void run() { while (!Thread.interrupted()) { - for (IncrementalIndex.TimeAndDims row : index.getFacts().keySet()) { + for (IncrementalIndexRow row : index.getFacts().keySet()) { if (index.getMetricLongValue(row.getRowIndex(), 0) != 1) { checkFailedCount.addAndGet(1); } diff --git a/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsCompTest.java b/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsCompTest.java deleted file mode 100644 index fa9490eae490..000000000000 --- a/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsCompTest.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.incremental; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import io.druid.data.input.MapBasedInputRow; -import io.druid.query.aggregation.CountAggregatorFactory; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Comparator; -import java.util.Map; - -import static io.druid.segment.incremental.IncrementalIndex.TimeAndDims; - -/** - */ -public class TimeAndDimsCompTest -{ - @Test - public void testBasic() - { - IncrementalIndex index = new IncrementalIndex.Builder() - .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) - .setMaxRowCount(1000) - .buildOnheap(); - - long time = System.currentTimeMillis(); - TimeAndDims td1 = index.toTimeAndDims(toMapRow(time, "billy", "A", "joe", "B")).getTimeAndDims(); - TimeAndDims td2 = index.toTimeAndDims(toMapRow(time, "billy", "A", "joe", "A")).getTimeAndDims(); - TimeAndDims td3 = index.toTimeAndDims(toMapRow(time, "billy", "A")).getTimeAndDims(); - - TimeAndDims td4 = index.toTimeAndDims(toMapRow(time + 1, "billy", "A", "joe", "B")).getTimeAndDims(); - TimeAndDims td5 = index.toTimeAndDims(toMapRow(time + 1, "billy", "A", "joe", Arrays.asList("A", "B"))).getTimeAndDims(); - TimeAndDims td6 = index.toTimeAndDims(toMapRow(time + 1)).getTimeAndDims(); - - Comparator comparator = index.dimsComparator(); - - Assert.assertEquals(0, comparator.compare(td1, td1)); - Assert.assertEquals(0, comparator.compare(td2, td2)); - Assert.assertEquals(0, comparator.compare(td3, td3)); - - Assert.assertTrue(comparator.compare(td1, td2) > 0); - Assert.assertTrue(comparator.compare(td2, td1) < 0); - Assert.assertTrue(comparator.compare(td2, td3) > 0); - Assert.assertTrue(comparator.compare(td3, td2) < 0); - Assert.assertTrue(comparator.compare(td1, td3) > 0); - Assert.assertTrue(comparator.compare(td3, td1) < 0); - - Assert.assertTrue(comparator.compare(td6, td1) > 0); - Assert.assertTrue(comparator.compare(td6, td2) > 0); - Assert.assertTrue(comparator.compare(td6, td3) > 0); - - Assert.assertTrue(comparator.compare(td4, td6) > 0); - Assert.assertTrue(comparator.compare(td5, td6) > 0); - Assert.assertTrue(comparator.compare(td4, td5) < 0); - Assert.assertTrue(comparator.compare(td5, td4) > 0); - } - - private MapBasedInputRow toMapRow(long time, Object... dimAndVal) - { - Map data = Maps.newHashMap(); - for (int i = 0; i < dimAndVal.length; i += 2) { - data.put((String) dimAndVal[i], dimAndVal[i + 1]); - } - return new MapBasedInputRow(time, Lists.newArrayList(data.keySet()), data); - } -} diff --git a/processing/src/test/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializerTest.java b/processing/src/test/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializerTest.java index acdf7df0e478..11ed71272db7 100644 --- a/processing/src/test/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializerTest.java @@ -26,6 +26,8 @@ import io.druid.java.util.common.io.smoosh.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.java.util.common.io.smoosh.SmooshedWriter; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.ObjectColumnSelector; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ComplexColumn; @@ -36,6 +38,7 @@ import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; @@ -51,7 +54,10 @@ public void testSanity() throws IOException HyperUniquesSerdeForTest serde = new HyperUniquesSerdeForTest(Hashing.murmur3_128()); int[] cases = {1000, 5000, 10000, 20000}; int[] columnSizes = { - Integer.MAX_VALUE, Integer.MAX_VALUE / 2, Integer.MAX_VALUE / 4, 5000 * Long.BYTES, + Integer.MAX_VALUE, + Integer.MAX_VALUE / 2, + Integer.MAX_VALUE / 4, + 5000 * Long.BYTES, 2500 * Long.BYTES }; @@ -71,7 +77,27 @@ public void testSanity() throws IOException byte[] hashBytes = fn.hashLong(i).asBytes(); collector.add(hashBytes); baseCollector.fold(collector); - serializer.serialize(collector); + serializer.serialize(new ObjectColumnSelector() + { + @Nullable + @Override + public Object getObject() + { + return collector; + } + + @Override + public Class classOfObject() + { + return HyperLogLogCollector.class; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // doesn't matter in tests + } + }); } try (final SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter( diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 5838f5a0025a..0b5c420badc1 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -469,20 +469,7 @@ public void testQueryWithInterval() throws InterruptedException realtimeManager3.start(); - while (true) { - boolean notAllStarted = realtimeManager3 - .getFireChiefs("testing").values().stream() - .anyMatch( - fireChief -> { - final Plumber plumber = fireChief.getPlumber(); - return plumber == null || !((TestPlumber) plumber).isStartedJob(); - } - ); - if (!notAllStarted) { - break; - } - Thread.sleep(10); - } + awaitStarted(); for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners((GroupByQueryRunnerFactory) factory)) { GroupByQuery query = GroupByQuery @@ -515,6 +502,24 @@ public void testQueryWithInterval() throws InterruptedException } + private void awaitStarted() throws InterruptedException + { + while (true) { + boolean notAllStarted = realtimeManager3 + .getFireChiefs("testing").values().stream() + .anyMatch( + fireChief -> { + final Plumber plumber = fireChief.getPlumber(); + return plumber == null || !((TestPlumber) plumber).isStartedJob(); + } + ); + if (!notAllStarted) { + break; + } + Thread.sleep(10); + } + } + @Test(timeout = 10_000L) public void testQueryWithSegmentSpec() throws InterruptedException { @@ -542,20 +547,7 @@ public void testQueryWithSegmentSpec() throws InterruptedException realtimeManager3.start(); - while (true) { - boolean notAllStarted = realtimeManager3 - .getFireChiefs("testing").values().stream() - .anyMatch( - fireChief -> { - final Plumber plumber = fireChief.getPlumber(); - return plumber == null || !((TestPlumber) plumber).isStartedJob(); - } - ); - if (!notAllStarted) { - break; - } - Thread.sleep(10); - } + awaitStarted(); for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners((GroupByQueryRunnerFactory) factory)) { GroupByQuery query = GroupByQuery @@ -654,20 +646,7 @@ public void testQueryWithMultipleSegmentSpec() throws InterruptedException realtimeManager3.start(); - while (true) { - boolean notAllStarted = realtimeManager3 - .getFireChiefs("testing").values().stream() - .anyMatch( - fireChief -> { - final Plumber plumber = fireChief.getPlumber(); - return plumber == null || !((TestPlumber) plumber).isStartedJob(); - } - ); - if (!notAllStarted) { - break; - } - Thread.sleep(10); - } + awaitStarted(); final Interval interval_26_28 = Intervals.of("2011-03-26T00:00:00.000Z/2011-03-28T00:00:00.000Z"); final Interval interval_28_29 = Intervals.of("2011-03-28T00:00:00.000Z/2011-03-29T00:00:00.000Z");