diff --git a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexIngestionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexIngestionBenchmark.java index 5d8b8e4671b4..530b3c7131aa 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexIngestionBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexIngestionBenchmark.java @@ -97,7 +97,7 @@ public void setup() throws IOException } } - @Setup(Level.Iteration) + @Setup(Level.Invocation) public void setup2() throws IOException { incIndex = makeIncIndex(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 80dc6891a720..bfa66838d2dd 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -1523,7 +1523,7 @@ private List readSegmentDim1(final SegmentDescriptor descriptor) throws ); IndexIO indexIO = new TestUtils().getTestIndexIO(); QueryableIndex index = indexIO.loadIndex(outputLocation); - DictionaryEncodedColumn dim1 = index.getColumn("dim1").getDictionaryEncoding(); + DictionaryEncodedColumn dim1 = index.getColumn("dim1").getDictionaryEncoding(); List values = Lists.newArrayList(); for (int i = 0; i < dim1.length(); i++) { int id = dim1.getSingleValueRow(i); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 2f48b8d4d3d0..2565e2f9e6fd 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -46,6 +46,8 @@ import io.druid.segment.BaseProgressIndicator; import io.druid.segment.ProgressIndicator; import io.druid.segment.QueryableIndex; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.OnheapIncrementalIndex; @@ -77,6 +79,7 @@ import java.util.Iterator; import java.util.LinkedHashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutionException; @@ -216,7 +219,8 @@ private static IncrementalIndex makeIncrementalIndex( Bucket theBucket, AggregatorFactory[] aggs, HadoopDruidIndexerConfig config, - Iterable oldDimOrder + Iterable oldDimOrder, + Map oldCapabilities ) { final HadoopTuningConfig tuningConfig = config.getSchema().getTuningConfig(); @@ -236,7 +240,7 @@ private static IncrementalIndex makeIncrementalIndex( ); if (oldDimOrder != null && !indexSchema.getDimensionsSpec().hasCustomDimensions()) { - newIndex.loadDimensionIterable(oldDimOrder); + newIndex.loadDimensionIterable(oldDimOrder, oldCapabilities); } return newIndex; @@ -340,7 +344,7 @@ protected void reduce( LinkedHashSet dimOrder = Sets.newLinkedHashSet(); SortableBytes keyBytes = SortableBytes.fromBytesWritable(key); Bucket bucket = Bucket.fromGroupKey(keyBytes.getGroupKey()).lhs; - IncrementalIndex index = makeIncrementalIndex(bucket, combiningAggs, config, null); + IncrementalIndex index = makeIncrementalIndex(bucket, combiningAggs, config, null, null); index.add(InputRowSerde.fromBytes(first.getBytes(), aggregators)); while (iter.hasNext()) { @@ -351,7 +355,7 @@ protected void reduce( dimOrder.addAll(index.getDimensionOrder()); log.info("current index full due to [%s]. creating new index.", index.getOutOfRowsReason()); flushIndexToContextAndClose(key, index, context); - index = makeIncrementalIndex(bucket, combiningAggs, config, dimOrder); + index = makeIncrementalIndex(bucket, combiningAggs, config, dimOrder, index.getColumnCapabilities()); } index.add(value); @@ -557,6 +561,7 @@ protected void reduce( bucket, combiningAggs, config, + null, null ); try { @@ -654,7 +659,8 @@ public void doRun() bucket, combiningAggs, config, - allDimensionNames + allDimensionNames, + persistIndex.getColumnCapabilities() ); startTime = System.currentTimeMillis(); ++indexCount; 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 20cf221a7f89..46bf466c8422 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java @@ -89,7 +89,8 @@ public InputRow get() IncrementalIndex.makeColumnSelectorFactory( aggFactory, supplier, - true + true, + null ) ); try { diff --git a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java index 84c444209935..ccfbfee55996 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java @@ -60,7 +60,7 @@ public Indexed getDimensionValues(String dimension) if (columnDesc == null || !columnDesc.getCapabilities().isDictionaryEncoded()) { return null; } - final DictionaryEncodedColumn column = columnDesc.getDictionaryEncoding(); + final DictionaryEncodedColumn column = columnDesc.getDictionaryEncoding(); return new Indexed() { @Override diff --git a/processing/src/main/java/io/druid/segment/DimensionHandler.java b/processing/src/main/java/io/druid/segment/DimensionHandler.java new file mode 100644 index 000000000000..c4472df2a46a --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DimensionHandler.java @@ -0,0 +1,211 @@ +/* + * 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.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.metamx.common.io.smoosh.FileSmoosher; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.GroupByQueryEngine; +import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.IOPeon; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; + +import java.io.Closeable; +import java.io.File; +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +/** + * Processing related interface + * + * A DimensionHandler is an object that encapsulates indexing, column merging/building, and querying operations + * for a given dimension type (e.g., dict-encoded String, Long). + * + * These operations are handled by sub-objects created through a DimensionHandler's methods: + * DimensionIndexer, DimensionMerger, and DimensionColumnReader, respectively. + * + * Each DimensionHandler object is associated with a single dimension. + * + * 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. + * + * A DimensionHandler is a stateless object, and thus thread-safe; its methods should be pure functions. + * + * The EncodedType and ActualType are Comparable because columns used as dimensions must have sortable values. + * + * @param class of the encoded values + * @param class of the actual values + */ +public interface DimensionHandler, EncodedTypeArray, ActualType extends Comparable> +{ + /** + * Get the name of the column associated with this handler. + * + * This string would be the output name of the column during ingestion, and the name of an input column when querying. + * + * @return Dimension name + */ + public String getDimensionName(); + + + /** + * Creates a new DimensionIndexer, a per-dimension object responsible for processing ingested rows in-memory, used by the + * IncrementalIndex. See {@link DimensionIndexer} interface for more information. + * + * @return A new DimensionIndexer object. + */ + public 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. + * + * See {@link DimensionMergerV9} interface for more information. + * + * @param indexSpec Specification object for the index merge + * @param outDir Location to store files generated by the merging process + * @param ioPeon ioPeon object passed in by IndexMerger, manages files created by the merging process + * @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler + * @param progress ProgressIndicator used by the merging process + + * @return A new DimensionMergerV9 object. + */ + public DimensionMergerV9 makeMerger( + IndexSpec indexSpec, + File outDir, + IOPeon ioPeon, + ColumnCapabilities capabilities, + ProgressIndicator progress + ); + + + /** + * Creates a new DimensionMergerLegacy, a per-dimension object responsible for merging indexes/row data across segments + * and building the on-disk representation of a dimension. For use with IndexMerger only. + * + * See {@link DimensionMergerLegacy} interface for more information. + * + * @param indexSpec Specification object for the index merge + * @param outDir Location to store files generated by the merging process + * @param ioPeon ioPeon object passed in by IndexMerger, manages files created by the merging process + * @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler + * @param progress ProgressIndicator used by the merging process + + * @return A new DimensionMergerLegacy object. + */ + public DimensionMergerLegacy makeLegacyMerger( + IndexSpec indexSpec, + File outDir, + IOPeon ioPeon, + ColumnCapabilities capabilities, + ProgressIndicator progress + ); + + + /** + * Given an array representing a single set of row value(s) for this dimension as an Object, + * return the length of the array after appropriate type-casting. + * + * For example, a dictionary encoded String dimension would receive an int[] as an Object. + * + * @param dimVals Array of row values + * @return Size of dimVals + */ + public int getLengthFromEncodedArray(EncodedTypeArray dimVals); + + + /** + * Given two arrays representing sorted encoded row value(s), return the result of their comparison. + * + * If the two arrays have different lengths, the shorter array should be ordered first in the comparison. + * + * Otherwise, this function should iterate through the array values and return the comparison of the first difference. + * + * @param lhs array of row values + * @param rhs array of row values + * + * @return integer indicating comparison result of arrays + */ + public int compareSortedEncodedArrays(EncodedTypeArray lhs, EncodedTypeArray rhs); + + + /** + * Given two arrays representing sorted encoded row value(s), check that the two arrays 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.validateSortedEncodedArrays() for a reference implementation. + * + * @param lhs array of row values + * @param rhs array of row values + * @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 + * + * @return integer indicating comparison result of arrays + */ + public void validateSortedEncodedArrays( + EncodedTypeArray lhs, + EncodedTypeArray rhs, + Indexed lhsEncodings, + Indexed rhsEncodings + ) throws SegmentValidationException; + + + /** + * 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. + */ + public Closeable getSubColumn(Column column); + + + /** + * Given a subcolumn from getSubColumn, and the index of the current row, retrieve a row as an array of values. + * + * 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 return the current row as a long[]. + * + * @param column Column for this dimension from a QueryableIndex + * @param currRow The index of the row to retrieve + * @return The row from "column" specified by "currRow", as an array of values + */ + public Object getRowValueArrayFromColumn(Closeable column, int currRow); +} diff --git a/processing/src/main/java/io/druid/segment/DimensionHandlerUtil.java b/processing/src/main/java/io/druid/segment/DimensionHandlerUtil.java new file mode 100644 index 000000000000..d265d111fdfc --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DimensionHandlerUtil.java @@ -0,0 +1,45 @@ +/* + * 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.Function; +import com.metamx.common.IAE; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ValueType; + +public final class DimensionHandlerUtil +{ + private DimensionHandlerUtil() {} + + public static DimensionHandler getHandlerFromCapabilities(String dimensionName, ColumnCapabilities capabilities) + { + DimensionHandler handler = null; + if (capabilities.getType() == ValueType.STRING) { + if (!capabilities.isDictionaryEncoded() || !capabilities.hasBitmapIndexes()) { + throw new IAE("String column must have dictionary encoding and bitmap index."); + } + handler = new StringDimensionHandler(dimensionName); + } + if (handler == null) { + throw new IAE("Could not create handler from invalid column type: " + capabilities.getType()); + } + return handler; + } +} diff --git a/processing/src/main/java/io/druid/segment/DimensionIndexer.java b/processing/src/main/java/io/druid/segment/DimensionIndexer.java new file mode 100644 index 000000000000..38e6235989e0 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DimensionIndexer.java @@ -0,0 +1,352 @@ +/* + * 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.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.MutableBitmap; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.filter.DruidPredicateFactory; +import io.druid.query.filter.ValueMatcher; +import io.druid.segment.data.Indexed; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexStorageAdapter; + +/** + * 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). + * + * 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. + * + * The DimensionIndexer is also responsible for implementing various value lookup operations, + * such as conversion between an encoded value and its full representation. It maintains knowledge of the + * mappings between encoded values and actual values. + * + * + * Sorting and Ordering + * -------------------- + * When encoding is present, there are two relevant orderings for the encoded values. + * + * 1.) Ordering based on encoded value's order of ingestion + * 2.) Ordering based on converted actual value + * + * Suppose we have a new String dimension DimA, which sees the values "Hello", "World", and "Apple", in that order. + * This would correspond to dictionary encodings of "Hello"=0, "World"=1, and "Apple"=2, by the order + * in which these values were first seen during ingestion. + * + * However, some use cases require the encodings to be sorted by their associated actual values. + * In this example, that ordering would be "Apple"=0, "Hello"=1, "World"=2. + * + * 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 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 + * persisted segments. + * + * Note that after calling the methods below that deal with sorted encodings, + * - getSortedEncodedValueFromUnsorted() + * - getUnsortedEncodedValueFromSorted() + * - getSortedIndexedValues() + * - convertUnsortedEncodedArrayToSortedEncodedArray() + * + * calling processRowValsToUnsortedEncodedArray() afterwards can invalidate previously read sorted encoding values + * (i.e., new values could be added that are inserted between existing values in the ordering). + * + * + * Thread Safety + * -------------------- + * Each DimensionIndexer exists within the context of a single IncrementalIndex. Before IndexMerger.persist() is + * called on an IncrementalIndex, any associated DimensionIndexers should allow multiple threads to add data to the + * indexer via processRowValsToUnsortedEncodedArray() and allow multiple threads to read data via methods that only + * deal with unsorted encodings. + * + * As mentioned in the "Sorting and Ordering" section, writes and calls to the sorted encoding + * methods should not be interleaved: the sorted encoding methods should only be called when it is known that + * writes to the indexer will no longer occur. + * + * The implementations of methods dealing with sorted encodings are free to assume that they will be called + * by only one thread. + * + * The sorted encoding methods are not currently used outside of index merging/persisting (single-threaded context, and + * no new events will be added to the indexer). + * + * If an indexer is passed to a thread that will use the sorted encoding methods, the caller is responsible + * for ensuring that previous writes to the indexer are visible to the thread that uses the sorted encoding space. + * + * For example, in the RealtimePlumber and IndexGeneratorJob, the thread that performs index persist is started + * by the same thread that handles the row adds on an index, ensuring the adds are visible to the persist thread. + * + * @param class of the encoded values + * @param class of the actual values + * + */ +public interface DimensionIndexer, EncodedTypeArray, ActualType extends Comparable> +{ + /** + * 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. + * + * For example, the dictionary-encoded String-type column will return an int[] containing a dictionary ID. + * + * The value within the returned array should be encoded if applicable, i.e. as instances of EncodedType. + * + * NOTE: This function can change the internal state of the DimensionIndexer. + * + * @param dimValues Single row val to process + * + * @return An array containing an encoded representation of the input row value. + */ + public EncodedTypeArray processRowValsToUnsortedEncodedArray(Object dimValues); + + + /** + * Given an encoded value that was ordered by time of ingestion, return the equivalent + * encoded value ordered by associated actual value. + * + * Using the example in the class description: + * getSortedEncodedValueFromUnsorted(0) would return 2 + * + * @param unsortedIntermediateValue value to convert + * @return converted value + */ + public EncodedType getSortedEncodedValueFromUnsorted(EncodedType unsortedIntermediateValue); + + + /** + * Given an encoded value that was ordered by associated actual value, return the equivalent + * encoded value ordered by time of ingestion. + * + * Using the example in the class description: + * getUnsortedEncodedValueFromSorted(2) would return 0 + * + * @param sortedIntermediateValue value to convert + * @return converted value + */ + public EncodedType getUnsortedEncodedValueFromSorted(EncodedType sortedIntermediateValue); + + + /** + * Returns an indexed structure of this dimension's sorted actual values. + * The integer IDs represent the ordering of the sorted values. + * + * Using the example in the class description: + * "Apple"=0, + * "Hello"=1, + * "World"=2 + * + * @return Sorted index of actual values + */ + public Indexed getSortedIndexedValues(); + + + /** + * Get the minimum dimension value seen by this indexer. + * + * NOTE: + * On an in-memory segment (IncrementaIndex), we can determine min/max values by looking at the stream of + * row values seen in calls to processSingleRowValToIndexKey(). + * + * However, on a disk-backed segment (QueryableIndex), the numeric dimensions do not currently have any + * supporting index structures that can be used to efficiently determine min/max values. + * + * When numeric dimension support is added, the segment format should be changed to store min/max values, to + * avoid performing a full-column scan to determine these values for numeric dims. + * + * @return min value + */ + public ActualType getMinValue(); + + + /** + * Get the maximum dimension value seen by this indexer. + * + * @return max value + */ + public ActualType getMaxValue(); + + + /** + * Get the cardinality of this dimension's values. + * + * @return value cardinality + */ + public int getCardinality(); + + + /** + * Return an object used to read rows from a StorageAdapter's Cursor. + * + * e.g. String -> DimensionSelector + * Long -> LongColumnSelector + * Float -> FloatColumnSelector + * + * See StringDimensionIndexer.makeColumnValueSelector() for a reference implementation. + * + * @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 desc Descriptor object for this dimension within an IncrementalIndex + * @return A new object that reads rows from currEntry + */ + public Object makeColumnValueSelector( + DimensionSpec spec, + IncrementalIndexStorageAdapter.EntryHolder currEntry, + IncrementalIndex.DimensionDesc desc + ); + + + /** + * Compares the row values for this DimensionIndexer's dimension from a TimeAndDims key. + * + * The dimension value arrays within a TimeAndDims 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. + * + * For example, a dictionary encoded String implementation would cast the Objects as int[] arrays. + * + * When comparing, if the two arrays have different lengths, the shorter array should be ordered first. + * + * Otherwise, the implementer of this function should iterate through the unsorted encoded values, converting + * them to their actual type (e.g., performing a dictionary lookup for a dict-encoded String dimension), + * and comparing the actual values until a difference is found. + * + * Refer to StringDimensionIndexer.compareUnsortedEncodedArrays() for a reference implementation. + * + * @param lhs dimension value array from a TimeAndDims key + * @param rhs dimension value array from a TimeAndDims key + * @return comparison of the two arrays + */ + public int compareUnsortedEncodedArrays(EncodedTypeArray lhs, EncodedTypeArray rhs); + + + /** + * Check if two row value arrays from TimeAndDims keys are equal. + * + * @param lhs dimension value array from a TimeAndDims key + * @param rhs dimension value array from a TimeAndDims key + * @return true if the two arrays are equal + */ + public boolean checkUnsortedEncodedArraysEqual(EncodedTypeArray lhs, EncodedTypeArray rhs); + + + /** + * Given a row value array from a TimeAndDims key, generate a hashcode. + * @param key dimension value array from a TimeAndDims key + * @return hashcode of the array + */ + public int getUnsortedEncodedArrayHashCode(EncodedTypeArray key); + + + /** + * Given a row value array from a TimeAndDims key, as described in the documentatiion for compareUnsortedEncodedArrays(), + * 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 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 + */ + public Object convertUnsortedEncodedArrayToActualArrayOrList(EncodedTypeArray key, boolean asList); + + + /** + * Given a row value array from a TimeAndDims key, as described in the documentatiion for compareUnsortedEncodedArrays(), + * 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 + */ + public EncodedTypeArray convertUnsortedEncodedArrayToSortedEncodedArray(EncodedTypeArray key); + + + /** + * 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", + * 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, + * and rowNum is 27, this function would set bit 27 in bitmapIndexes[1], bitmapIndexes[3], and bitmapIndexes[4] + * + * See StringDimensionIndexer.fillBitmapsFromUnsortedEncodedArray() for a reference implementation. + * + * 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 rowNum current row number + * @param bitmapIndexes array of bitmaps, indexed by integer dimension value + * @param factory bitmap factory + */ + public void fillBitmapsFromUnsortedEncodedArray(EncodedTypeArray key, int rowNum, MutableBitmap[] bitmapIndexes, BitmapFactory factory); + + + /** + * Return a ValueMatcher that accepts an EntryHolder containing the current TimeAndDims key and the array index of this + * indexer's dimension within the TimeAndDims key. + * + * The implementer should read the dimension array Object from the TimeAndDims key and cast it to the appropriate + * type, as described in the documentation for compareUnsortedEncodedArrays(). + * + * The returned ValueMatcher should match the dimension values against matchValue. + * + * See StringDimensionIndexer for a reference implementation. + * + * @param matchValue value to match on + * @param holder holds the current TimeAndDims key during row iteration + * @param dimIndex the array index of this indexer's dimension within the TimeAndDims key + * @return A ValueMatcher that matches a dimension value array from a TimeAndDims key against "matchValue" + */ + public ValueMatcher makeIndexingValueMatcher(Comparable matchValue, IncrementalIndexStorageAdapter.EntryHolder holder, int dimIndex); + + + /** + * Return a ValueMatcher that accepts an EntryHolder containing the current TimeAndDims key and the array index of this + * indexer's dimension within the TimeAndDims key. + * + * The implementer should read the dimension array Object from the TimeAndDims key and cast it to the appropriate + * type, as described in the documentation for compareUnsortedEncodedArrays(). + * + * Based on the type of the indexer, this method should get a predicate of the same type from the supplied + * predicateFactory. + * + * For example, a StringDimensionIndexer would call predicateFactory.makeStringPredicate(). + * + * The returned ValueMatcher should apply the generated predicate to the dimension values. + * + * See StringDimensionIndexer for a reference implementation. + * + * @param predicateFactory Factory object that can generate predicates for each supported dimension type + * @param holder holds the current TimeAndDims key during row iteration + * @param dimIndex the array index of this indexer's dimension within the TimeAndDims key + * @return A ValueMatcher that applies a predicate from the predicateFactory to the dimension values in the TimeAndDim keys + */ + public ValueMatcher makeIndexingValueMatcher(DruidPredicateFactory predicateFactory, IncrementalIndexStorageAdapter.EntryHolder holder, int dimIndex); +} diff --git a/processing/src/main/java/io/druid/segment/DimensionMerger.java b/processing/src/main/java/io/druid/segment/DimensionMerger.java new file mode 100644 index 000000000000..4dbf406abe70 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DimensionMerger.java @@ -0,0 +1,139 @@ +/* + * 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.io.Closer; + +import java.io.IOException; +import java.nio.IntBuffer; +import java.util.List; + +/** + * 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). + * + * This object is responsible for: + * - merging any relevant structures from the segments (e.g., encoding dictionaries) + * - 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 the rows across segments into a common sequence of rows + * + * - After constructing the merged sequence of rows, build any applicable index structures (e.g, bitmap indexes) + * + * - Write the value representation metadata (e.g. dictionary), the sequence of row values, + * and index structures to a merged segment. + * + * A class implementing this interface is expected to be highly stateful, updating its internal state as these + * functions are called. + */ +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. + * + * 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. + * + * @param adapters List of adapters to be merged. + * @throws IOException + */ + public void writeMergedValueMetadata(List adapters) throws IOException; + + + /** + * Convert a row from a single segment to its equivalent representation in the merged set of rows. + * + * This function 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. + * + * 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(). + * + * @param segmentRow A row from a segment to be converted to its representation within the merged sequence of rows. + * @param segmentIndexNumber Integer indicating which segment the row originated from. + */ + public EncodedTypedArray convertSegmentRowValuesToMergedRowValues(EncodedTypedArray segmentRow, int segmentIndexNumber); + + + /** + * Process a row from the merged sequence of rows 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 values from each dimension to their correspodning DimensionMergers. + * + * 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 + */ + public void processMergedRow(EncodedTypedArray rowValues) throws IOException; + + + /** + * Internally construct any index structures relevant to this DimensionMerger. + * + * After receiving the sequence of merged rows via iterated processMergedRow() calls, the DimensionMerger + * can now build any index structures it needs. + * + * For example, a dictionary encoded String implementation would create its bitmap indexes + * for the merged segment during this step. + * + * 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(). + * + * 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. + * @param closer Add Closeables for resource cleanup to this Closer if needed + * @throws IOException + */ + public void writeIndexes(List segmentRowNumConversions, Closer closer) throws IOException; + + + /** + * Return true if this dimension's data does not need to be written to the segment. + * + * For example, if a dictionary-encoded String dimension had only null values, it can be skipped. + * + * @return true if this dimension can be excluded from the merged segment. + */ + public boolean canSkip(); +} diff --git a/processing/src/main/java/io/druid/segment/DimensionMergerLegacy.java b/processing/src/main/java/io/druid/segment/DimensionMergerLegacy.java new file mode 100644 index 000000000000..360eb8672b2f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DimensionMergerLegacy.java @@ -0,0 +1,65 @@ +/* + * 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.io.ByteSink; +import com.google.common.io.OutputSupplier; +import io.druid.common.guava.FileOutputSupplier; + +import java.io.FileOutputStream; +import java.io.IOException; + +/** + * Processing related interface + * + * DimensionMerger subclass to be used with the legacy IndexMerger. + * + * NOTE: Remove this class when the legacy IndexMerger is deprecated and removed. + */ +public interface DimensionMergerLegacy extends DimensionMergerV9 +{ + /** + * Write this dimension's value metadata to a file. + * + * @param valueEncodingFile Destination file + * @throws IOException + */ + public void writeValueMetadataToFile(FileOutputSupplier valueEncodingFile) throws IOException; + + + /** + * Write this dimension's sequence of row values to a file. + * @param rowValueOut Destination file + * @throws IOException + */ + public void writeRowValuesToFile(FileOutputSupplier rowValueOut) throws IOException; + + + /** + * Write this dimension's bitmap and spatial indexes to a file. + * @param invertedOut Destination file for bitmap indexes + * @param spatialOut Destination file for spatial indexes + * @throws IOException + */ + public void writeIndexesToFiles( + ByteSink invertedOut, + OutputSupplier spatialOut + ) throws IOException; +} diff --git a/processing/src/main/java/io/druid/segment/DimensionMergerV9.java b/processing/src/main/java/io/druid/segment/DimensionMergerV9.java new file mode 100644 index 000000000000..c344c84c5404 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DimensionMergerV9.java @@ -0,0 +1,38 @@ +/* + * 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.column.ColumnDescriptor; + +/** + * Processing related interface + * + * DimensionMerger subclass to be used with IndexMergerV9. + */ +public interface DimensionMergerV9 extends DimensionMerger +{ + /** + * Return a ColumnDescriptor containing ColumnPartSerde objects appropriate for + * this DimensionMerger's value metadata, sequence of row values, and index structures. + * + * @return ColumnDescriptor that IndexMergerV9 will use to build a column. + */ + public ColumnDescriptor makeColumnDescriptor(); +} diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 0c6ad9f10585..d88984799de9 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -52,6 +52,7 @@ import io.druid.common.utils.SerializerUtils; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; @@ -66,6 +67,7 @@ import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedIterable; +import io.druid.segment.data.IndexedLongs; import io.druid.segment.data.IndexedMultivalue; import io.druid.segment.data.IndexedRTree; import io.druid.segment.data.VSizeIndexed; @@ -87,6 +89,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.lang.reflect.Array; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.AbstractList; @@ -173,6 +176,8 @@ 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(); long row = 0L; @@ -188,7 +193,7 @@ public void validateTwoSegments(final IndexableAdapter adapter1, final Indexable } if (rb1.compareTo(rb2) != 0) { try { - validateRowValues(rb1, adapter1, rb2, adapter2); + validateRowValues(dimHandlers, rb1, adapter1, rb2, adapter2); } catch (SegmentValidationException ex) { throw new SegmentValidationException(ex, "Validation failure on row %d: [%s] vs [%s]", row, rb1, rb2); @@ -307,6 +312,7 @@ static interface IndexIOHandler } public static void validateRowValues( + Map dimHandlers, Rowboat rb1, IndexableAdapter adapter1, Rowboat rb2, @@ -319,8 +325,8 @@ public static void validateRowValues( rb1.getTimestamp(), rb2.getTimestamp() ); } - final int[][] dims1 = rb1.getDims(); - final int[][] dims2 = rb2.getDims(); + final Object[] dims1 = rb1.getDims(); + final Object[] dims2 = rb2.getDims(); if (dims1.length != dims2.length) { throw new SegmentValidationException( "Dim lengths not equal %s vs %s", @@ -331,90 +337,31 @@ public static void validateRowValues( final Indexed dim1Names = adapter1.getDimensionNames(); final Indexed dim2Names = adapter2.getDimensionNames(); for (int i = 0; i < dims1.length; ++i) { - final int[] dim1Vals = dims1[i]; - final int[] dim2Vals = dims2[i]; + final Object dim1Vals = dims1[i]; + final Object dim2Vals = dims2[i]; final String dim1Name = dim1Names.get(i); final String dim2Name = dim2Names.get(i); - final Indexed dim1ValNames = adapter1.getDimValueLookup(dim1Name); - final Indexed dim2ValNames = adapter2.getDimValueLookup(dim2Name); - - if (dim1Vals == null || dim2Vals == null) { - if (dim1Vals != dim2Vals) { - throw new SegmentValidationException( - "Expected nulls, found %s and %s", - Arrays.toString(dim1Vals), - Arrays.toString(dim2Vals) - ); - } else { - continue; - } - } - if (dim1Vals.length != dim2Vals.length) { - // Might be OK if one of them has null. This occurs in IndexMakerTest - if (dim1Vals.length == 0 && dim2Vals.length == 1) { - final String dimValName = dim2ValNames.get(dim2Vals[0]); - if (dimValName == null) { - continue; - } else { - throw new SegmentValidationException( - "Dim [%s] value [%s] is not null", - dim2Name, - dimValName - ); - } - } else if (dim2Vals.length == 0 && dim1Vals.length == 1) { - final String dimValName = dim1ValNames.get(dim1Vals[0]); - if (dimValName == null) { - continue; - } else { - throw new SegmentValidationException( - "Dim [%s] value [%s] is not null", - dim1Name, - dimValName - ); - } - } else { - throw new SegmentValidationException( - "Dim [%s] value lengths not equal. Expected %d found %d", - dim1Name, - dims1.length, - dims2.length - ); - } - } - - for (int j = 0; j < Math.max(dim1Vals.length, dim2Vals.length); ++j) { - final int dIdex1 = dim1Vals.length <= j ? -1 : dim1Vals[j]; - final int dIdex2 = dim2Vals.length <= j ? -1 : dim2Vals[j]; - if (dIdex1 == dIdex2) { - continue; - } - - final String dim1ValName = dIdex1 < 0 ? null : dim1ValNames.get(dIdex1); - final String dim2ValName = dIdex2 < 0 ? null : dim2ValNames.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]", - dim1Name, - dim1ValName, - dim2ValName - ); - } - } - - if (!dim1ValName.equals(dim2ValName)) { - throw new SegmentValidationException( - "Dim [%s] value not equal. Expected [%s] found [%s]", - dim1Name, - dim1ValName, - dim2ValName - ); - } + ColumnCapabilities capabilities1 = adapter1.getCapabilities(dim1Name); + ColumnCapabilities capabilities2 = adapter2.getCapabilities(dim2Name); + ValueType dim1Type = capabilities1.getType(); + ValueType dim2Type = capabilities2.getType(); + if (dim1Type != dim2Type) { + throw new SegmentValidationException( + "Dim [%s] types not equal. Expected %d found %d", + dim1Name, + dim1Type, + dim2Type + ); } + + DimensionHandler dimHandler = dimHandlers.get(dim1Name); + dimHandler.validateSortedEncodedArrays( + dim1Vals, + dim2Vals, + adapter1.getDimValueLookup(dim1Name), + adapter2.getDimValueLookup(dim2Name) + ); } } @@ -748,7 +695,10 @@ public int size() channel.write(ByteBuffer.wrap(specBytes)); serdeficator.write(channel); channel.close(); - } else if (filename.startsWith("met_")) { + } else if (filename.startsWith("met_") || filename.startsWith("numeric_dim_")) { + // NOTE: identifying numeric dimensions by using a different filename pattern is meant to allow the + // legacy merger (which will be deprecated) to support long/float dims. Going forward, the V9 merger + // should be used instead if any dimension types beyond String are needed. if (!filename.endsWith(String.format("%s.drd", BYTE_ORDER))) { skippedFiles.add(filename); continue; @@ -1101,6 +1051,11 @@ public static File makeDimFile(File dir, String dimension) return new File(dir, String.format("dim_%s.drd", dimension)); } + public static File makeNumericDimFile(File dir, String dimension, ByteOrder order) + { + return new File(dir, String.format("numeric_dim_%s_%s.drd", dimension, order)); + } + public static File makeTimeFile(File dir, ByteOrder order) { return new File(dir, String.format("time_%s.drd", order)); diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index cc73cd6078c2..8fd2ecb87207 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.base.Preconditions; import com.google.common.base.Splitter; import com.google.common.base.Strings; -import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; @@ -34,7 +33,6 @@ import com.google.common.collect.PeekingIterator; import com.google.common.collect.Sets; import com.google.common.io.ByteSink; -import com.google.common.io.ByteStreams; import com.google.common.io.Closer; import com.google.common.io.FileWriteMode; import com.google.common.io.Files; @@ -42,13 +40,6 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.google.inject.Inject; -import com.metamx.collections.bitmap.BitmapFactory; -import com.metamx.collections.bitmap.ImmutableBitmap; -import com.metamx.collections.bitmap.MutableBitmap; -import com.metamx.collections.spatial.ImmutableRTree; -import com.metamx.collections.spatial.RTree; -import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy; -import com.metamx.common.ByteBufferUtils; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.Pair; @@ -67,20 +58,16 @@ import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ValueType; import io.druid.segment.data.BitmapSerdeFactory; -import io.druid.segment.data.ByteBufferWriter; import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.LongSupplierSerializer; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedIterable; -import io.druid.segment.data.IndexedRTree; import io.druid.segment.data.ListIndexed; import io.druid.segment.data.TmpFileIOPeon; -import io.druid.segment.data.VSizeIndexedWriter; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; import io.druid.segment.serde.ComplexMetricColumnSerializer; @@ -97,7 +84,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.IntBuffer; -import java.nio.MappedByteBuffer; import java.nio.channels.FileChannel; import java.util.ArrayList; import java.util.Arrays; @@ -611,6 +597,7 @@ public Metadata apply(IndexableAdapter input) final Map valueTypes = Maps.newTreeMap(Ordering.natural().nullsFirst()); final Map metricTypeNames = Maps.newTreeMap(Ordering.natural().nullsFirst()); final Map columnCapabilities = Maps.newHashMap(); + final List dimCapabilities = new ArrayList<>(); for (IndexableAdapter adapter : indexes) { for (String dimension : adapter.getDimensionNames()) { @@ -618,7 +605,6 @@ public Metadata apply(IndexableAdapter input) ColumnCapabilities capabilities = adapter.getCapabilities(dimension); if (mergedCapabilities == null) { mergedCapabilities = new ColumnCapabilitiesImpl(); - mergedCapabilities.setType(ValueType.STRING); } columnCapabilities.put(dimension, mergedCapabilities.merge(capabilities)); } @@ -635,6 +621,10 @@ public Metadata apply(IndexableAdapter input) } } + for (String dimension : mergedDimensions) { + dimCapabilities.add(columnCapabilities.get(dimension)); + } + Closer closer = Closer.create(); final Interval dataInterval; final File v8OutDir = new File(outDir, "v8-tmp"); @@ -689,93 +679,22 @@ public void close() throws IOException startTime = System.currentTimeMillis(); final ArrayList dimOuts = Lists.newArrayListWithCapacity(mergedDimensions.size()); - final Map dimensionCardinalities = Maps.newHashMap(); - final ArrayList> dimConversions = Lists.newArrayListWithCapacity(indexes.size()); - final ArrayList convertMissingDimsFlags = Lists.newArrayListWithCapacity(mergedDimensions.size()); - final ArrayList nullRowsList = Lists.newArrayListWithCapacity(mergedDimensions.size()); - final ArrayList dimHasNullFlags = Lists.newArrayListWithCapacity(mergedDimensions.size()); - - for (int i = 0; i < indexes.size(); ++i) { - dimConversions.add(Maps.newHashMap()); - } - - for (String dimension : mergedDimensions) { - nullRowsList.add(indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap()); - - final GenericIndexedWriter writer = new GenericIndexedWriter( - ioPeon, dimension, GenericIndexed.STRING_STRATEGY + final DimensionHandler[] handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities); + final List mergers = new ArrayList<>(); + for (int i = 0; i < mergedDimensions.size(); i++) { + DimensionMergerLegacy merger = handlers[i].makeLegacyMerger( + indexSpec, + v8OutDir, + ioPeon, + dimCapabilities.get(i), + progress ); - writer.open(); - - boolean dimHasNull = false; - boolean dimHasValues = false; - boolean dimAbsentFromSomeIndex = false; - - int numMergeIndex = 0; - Indexed dimValueLookup = null; - Indexed[] dimValueLookups = new Indexed[indexes.size() + 1]; - for (int i = 0; i < indexes.size(); i++) { - Indexed dimValues = indexes.get(i).getDimValueLookup(dimension); - if (!isNullColumn(dimValues)) { - dimHasValues = true; - dimHasNull |= dimValues.indexOf(null) >= 0; - dimValueLookups[i] = dimValueLookup = dimValues; - numMergeIndex++; - } else { - dimAbsentFromSomeIndex = true; - } - } - - boolean convertMissingDims = dimHasValues && dimAbsentFromSomeIndex; - convertMissingDimsFlags.add(convertMissingDims); - - /* - * 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 - * 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. - */ - if (convertMissingDims && !dimHasNull) { - dimHasNull = true; - dimValueLookups[indexes.size()] = dimValueLookup = EMPTY_STR_DIM_VAL; - numMergeIndex++; - } - - int cardinality = 0; - if (numMergeIndex > 1) { - DictionaryMergeIterator iterator = new DictionaryMergeIterator(dimValueLookups, true); - - while (iterator.hasNext()) { - writer.write(iterator.next()); - } + mergers.add(merger); + merger.writeMergedValueMetadata(indexes); - for (int i = 0; i < indexes.size(); i++) { - if (dimValueLookups[i] != null && iterator.needConversion(i)) { - dimConversions.get(i).put(dimension, iterator.conversions[i]); - } - } - cardinality = iterator.counter; - } else if (numMergeIndex == 1) { - for (String value : dimValueLookup) { - writer.write(value); - } - cardinality = dimValueLookup.size(); - } - - dimensionCardinalities.put(dimension, cardinality); - - // Mark if this dim has the null/empty str value in its dictionary, used for determining nullRowsList later. - dimHasNullFlags.add(dimHasNull); - - FileOutputSupplier dimOut = new FileOutputSupplier(IndexIO.makeDimFile(v8OutDir, dimension), true); + FileOutputSupplier dimOut = new FileOutputSupplier(IndexIO.makeDimFile(v8OutDir, mergedDimensions.get(i)), true); + merger.writeValueMetadataToFile(dimOut); dimOuts.add(dimOut); - - writer.close(); - serializerUtils.writeString(dimOut, dimension); - ByteStreams.copy(writer.combineStreams(), dimOut); - - ioPeon.cleanup(); } log.info("outDir[%s] completed dim conversions in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); @@ -787,9 +706,10 @@ public void close() throws IOException indexes, mergedDimensions, mergedMetrics, - dimConversions, - convertMissingDimsFlags, - rowMergerFn + rowMergerFn, + dimCapabilities, + handlers, + mergers ); LongSupplierSerializer timeWriter = CompressionFactory.getLongSerializer( @@ -799,13 +719,6 @@ public void close() throws IOException timeWriter.open(); - ArrayList forwardDimWriters = Lists.newArrayListWithCapacity(mergedDimensions.size()); - for (String dimension : mergedDimensions) { - VSizeIndexedWriter writer = new VSizeIndexedWriter(ioPeon, dimension, dimensionCardinalities.get(dimension)); - writer.open(); - forwardDimWriters.add(writer); - } - ArrayList metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding(); @@ -855,20 +768,9 @@ public void close() throws IOException metWriters.get(i).serialize(metrics[i]); } - int[][] dims = theRow.getDims(); + Object[] dims = theRow.getDims(); for (int i = 0; i < dims.length; ++i) { - List listToWrite = (i >= dims.length || dims[i] == null) - ? null - : Ints.asList(dims[i]); - forwardDimWriters.get(i).write(listToWrite); - if (listToWrite == null || listToWrite.isEmpty()) { - // empty row; add to the nullRows bitmap - nullRowsList.get(i).add(rowCount); - } else if (dimHasNullFlags.get(i) && listToWrite.size() == 1 && listToWrite.get(0) == 0) { - // 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 nullRowsList. - nullRowsList.get(i).add(rowCount); - } + mergers.get(i).processMergedRow(dims[i]); } for (Map.Entry> comprisedRow : theRow.getComprisedRows().entrySet()) { @@ -900,16 +802,10 @@ public void close() throws IOException timeWriter.closeAndConsolidate(out); IndexIO.checkFileSize(timeFile); - for (int i = 0; i < mergedDimensions.size(); ++i) { - forwardDimWriters.get(i).close(); - ByteStreams.copy(forwardDimWriters.get(i).combineStreams(), dimOuts.get(i)); - } - for (MetricColumnSerializer metWriter : metWriters) { metWriter.close(); } - ioPeon.cleanup(); log.info( "outDir[%s] completed walk through of %,d rows in %,d millis.", v8OutDir, @@ -917,7 +813,7 @@ public void close() throws IOException System.currentTimeMillis() - startTime ); - /************ Create Inverted Indexes *************/ + /************ Create Inverted Indexes and Finalize Columns *************/ startTime = System.currentTimeMillis(); final File invertedFile = new File(v8OutDir, "inverted.drd"); @@ -928,119 +824,35 @@ public void close() throws IOException Files.touch(geoFile); OutputSupplier spatialOut = Files.newOutputStreamSupplier(geoFile, true); - for (int i = 0; i < mergedDimensions.size(); ++i) { - long dimStartTime = System.currentTimeMillis(); - String dimension = mergedDimensions.get(i); + for (int i = 0; i < mergedDimensions.size(); i++) { + DimensionMergerLegacy legacyMerger = (DimensionMergerLegacy) mergers.get(i); + legacyMerger.writeIndexes(rowNumConversions, closer); + legacyMerger.writeIndexesToFiles(out, spatialOut); + legacyMerger.writeRowValuesToFile(dimOuts.get(i)); + } + log.info("outDir[%s] completed inverted.drd and wrote dimensions in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); - File dimOutFile = dimOuts.get(i).getFile(); - final MappedByteBuffer dimValsMapped = Files.map(dimOutFile); - closer.register(new Closeable() + final Function dimFilenameFunction = new Function() + { + @Override + public String apply(@Nullable String input) { - @Override - public void close() throws IOException - { - ByteBufferUtils.unmap(dimValsMapped); - } - }); - - if (!dimension.equals(serializerUtils.readString(dimValsMapped))) { - throw new ISE("dimensions[%s] didn't equate!? This is a major WTF moment.", dimension); - } - Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY); - log.info("Starting dimension[%s] with cardinality[%,d]", dimension, dimVals.size()); - - final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); - GenericIndexedWriter writer = new GenericIndexedWriter<>( - ioPeon, dimension, bitmapSerdeFactory.getObjectStrategy() - ); - writer.open(); - - boolean isSpatialDim = columnCapabilities.get(dimension).hasSpatialIndexes(); - ByteBufferWriter spatialWriter = null; - RTree tree = null; - IOPeon spatialIoPeon = new TmpFileIOPeon(); - if (isSpatialDim) { - BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory(); - spatialWriter = new ByteBufferWriter( - spatialIoPeon, dimension, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapFactory) - ); - spatialWriter.open(); - tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory); - } - - IndexSeeker[] dictIdSeeker = toIndexSeekers(indexes, dimConversions, dimension); - - //Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result. - for (int dictId = 0; dictId < dimVals.size(); dictId++) { - progress.progress(); - List> convertedInverteds = Lists.newArrayListWithCapacity(indexes.size()); - for (int j = 0; j < indexes.size(); ++j) { - int seekedDictId = dictIdSeeker[j].seek(dictId); - if (seekedDictId != IndexSeeker.NOT_EXIST) { - convertedInverteds.add( - new ConvertingIndexedInts( - indexes.get(j).getBitmapIndex(dimension, seekedDictId), rowNumConversions.get(j) - ) - ); - } - } - - MutableBitmap bitset = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap(); - for (Integer row : CombiningIterable.createSplatted( - convertedInverteds, - Ordering.natural().nullsFirst() - )) { - if (row != INVALID_ROW) { - bitset.add(row); - } - } - - if ((dictId == 0) && (Iterables.getFirst(dimVals, "") == null)) { - bitset.or(nullRowsList.get(i)); - } - - writer.write( - bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(bitset) - ); - - if (isSpatialDim) { - String dimVal = dimVals.get(dictId); - if (dimVal != null) { - List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); - float[] coords = new float[stringCoords.size()]; - for (int j = 0; j < coords.length; j++) { - coords[j] = Float.valueOf(stringCoords.get(j)); - } - tree.insert(coords, bitset); - } + String formatString; + if (columnCapabilities.get(input).isDictionaryEncoded()) { + formatString = "dim_%s.drd"; + } else { + formatString = String.format("numeric_dim_%%s_%s.drd", IndexIO.BYTE_ORDER); } + return GuavaUtils.formatFunction(formatString).apply(input); } - writer.close(); - - serializerUtils.writeString(out, dimension); - ByteStreams.copy(writer.combineStreams(), out); - ioPeon.cleanup(); - - log.info("Completed dimension[%s] in %,d millis.", dimension, System.currentTimeMillis() - dimStartTime); - - if (isSpatialDim) { - spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); - spatialWriter.close(); - - serializerUtils.writeString(spatialOut, dimension); - ByteStreams.copy(spatialWriter.combineStreams(), spatialOut); - spatialIoPeon.cleanup(); - } - } - - log.info("outDir[%s] completed inverted.drd in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); + }; final ArrayList expectedFiles = Lists.newArrayList( Iterables.concat( Arrays.asList( "index.drd", "inverted.drd", "spatial.drd", String.format("time_%s.drd", IndexIO.BYTE_ORDER) ), - Iterables.transform(mergedDimensions, GuavaUtils.formatFunction("dim_%s.drd")), + Iterables.transform(mergedDimensions, dimFilenameFunction), Iterables.transform( mergedMetrics, GuavaUtils.formatFunction(String.format("met_%%s_%s.drd", IndexIO.BYTE_ORDER)) ) @@ -1095,13 +907,25 @@ public void close() throws IOException } } + protected DimensionHandler[] makeDimensionHandlers(final List mergedDimensions, final List dimCapabilities) + { + final DimensionHandler[] handlers = new DimensionHandler[mergedDimensions.size()]; + for (int i = 0; i < mergedDimensions.size(); i++) { + ColumnCapabilities capabilities = dimCapabilities.get(i); + String dimName = mergedDimensions.get(i); + handlers[i] = DimensionHandlerUtil.getHandlerFromCapabilities(dimName, capabilities); + } + return handlers; + } + protected Iterable makeRowIterable( List indexes, final List mergedDimensions, final List mergedMetrics, - ArrayList> dimConversions, - ArrayList convertMissingDimsFlags, - Function>, Iterable> rowMergerFn + Function>, Iterable> rowMergerFn, + final List dimCapabilities, + final DimensionHandler[] handlers, + final List mergers ) { ArrayList> boats = Lists.newArrayListWithCapacity(indexes.size()); @@ -1109,8 +933,8 @@ protected Iterable makeRowIterable( for (int i = 0; i < indexes.size(); ++i) { final IndexableAdapter adapter = indexes.get(i); - final int[] dimLookup = toLookupMap(adapter.getDimensionNames(), mergedDimensions); - final int[] metricLookup = toLookupMap(adapter.getMetricNames(), mergedMetrics); + 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) { @@ -1122,14 +946,18 @@ protected Iterable makeRowIterable( @Override public Rowboat apply(Rowboat input) { - int[][] newDims = input.getDims(); + Object[] newDims; if (dimLookup != null) { - newDims = new int[mergedDimensions.size()][]; + newDims = new Object[mergedDimensions.size()]; int j = 0; - for (int[] dim : input.getDims()) { + 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(); @@ -1146,7 +974,8 @@ public Rowboat apply(Rowboat input) input.getTimestamp(), newDims, newMetrics, - input.getRowNum() + input.getRowNum(), + handlers ); } } @@ -1154,7 +983,7 @@ public Rowboat apply(Rowboat input) } boats.add( new MMappedIndexRowIterable( - target, mergedDimensions, dimConversions.get(i), i, convertMissingDimsFlags + target, mergedDimensions, i, dimCapabilities, mergers ) ); } @@ -1162,14 +991,16 @@ public Rowboat apply(Rowboat input) return rowMergerFn.apply(boats); } - private int[] toLookupMap(Indexed indexed, List values) + // 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) { - if (isSame(indexed, values)) { - return null; // no need to convert + if (isSame(adapterColumnNames, mergedColumnNames)) { + return null; // no need to convert if column lists are identical } - int[] dimLookup = new int[values.size()]; - for (int i = 0; i < indexed.size(); i++) { - dimLookup[i] = values.indexOf(indexed.get(i)); + int[] dimLookup = new int[mergedColumnNames.size()]; + for (int i = 0; i < adapterColumnNames.size(); i++) { + dimLookup[i] = mergedColumnNames.indexOf(adapterColumnNames.get(i)); } return dimLookup; } @@ -1226,25 +1057,6 @@ public void createIndexDrdFile( IndexIO.checkFileSize(indexFile); } - protected IndexSeeker[] toIndexSeekers( - List adapters, - ArrayList> dimConversions, - String dimension - ) - { - IndexSeeker[] seekers = new IndexSeeker[adapters.size()]; - for (int i = 0; i < adapters.size(); i++) { - IntBuffer dimConversion = dimConversions.get(i).get(dimension); - if (dimConversion != null) { - seekers[i] = new IndexSeekerWithConversion((IntBuffer) dimConversion.asReadOnlyBuffer().rewind()); - } else { - Indexed dimValueLookup = adapters.get(i).getDimValueLookup(dimension); - seekers[i] = new IndexSeekerWithoutConversion(dimValueLookup == null ? 0 : dimValueLookup.size()); - } - } - return seekers; - } - static interface IndexSeeker { int NOT_EXIST = -1; @@ -1253,22 +1065,6 @@ static interface IndexSeeker int seek(int dictId); } - static class IndexSeekerWithoutConversion implements IndexSeeker - { - private final int limit; - - public IndexSeekerWithoutConversion(int limit) - { - this.limit = limit; - } - - @Override - public int seek(int dictId) - { - return dictId < limit ? dictId : NOT_EXIST; - } - } - /** * Get old dictId from new dictId, and only support access in order */ @@ -1324,69 +1120,28 @@ public int seek(int dictId) } } - public static class ConvertingIndexedInts implements Iterable - { - private final IndexedInts baseIndex; - private final IntBuffer conversionBuffer; - - public ConvertingIndexedInts( - IndexedInts baseIndex, - IntBuffer conversionBuffer - ) - { - this.baseIndex = baseIndex; - this.conversionBuffer = conversionBuffer; - } - - public int size() - { - return baseIndex.size(); - } - - public int get(int index) - { - return conversionBuffer.get(baseIndex.get(index)); - } - - @Override - public Iterator iterator() - { - return Iterators.transform( - baseIndex.iterator(), - new Function() - { - @Override - public Integer apply(@Nullable Integer input) - { - return conversionBuffer.get(input); - } - } - ); - } - } - public static class MMappedIndexRowIterable implements Iterable { private final Iterable index; private final List convertedDims; - private final Map converters; private final int indexNumber; - private final ArrayList convertMissingDimsFlags; - private static final int[] EMPTY_STR_DIM = new int[]{0}; + private final List dimCapabilities; + private final List mergers; + MMappedIndexRowIterable( Iterable index, List convertedDims, - Map converters, int indexNumber, - ArrayList convertMissingDimsFlags + final List dimCapabilities, + final List mergers ) { this.index = index; this.convertedDims = convertedDims; - this.converters = converters; this.indexNumber = indexNumber; - this.convertMissingDimsFlags = convertMissingDimsFlags; + this.dimCapabilities = dimCapabilities; + this.mergers = mergers; } public Iterable getIndex() @@ -1397,18 +1152,6 @@ public Iterable getIndex() @Override public Iterator iterator() { - final IntBuffer[] converterArray = FluentIterable - .from(convertedDims) - .transform( - new Function() - { - @Override - public IntBuffer apply(String input) - { - return converters.get(input); - } - } - ).toArray(IntBuffer.class); return Iterators.transform( index.iterator(), new Function() @@ -1416,36 +1159,21 @@ public IntBuffer apply(String input) @Override public Rowboat apply(@Nullable Rowboat input) { - int[][] dims = input.getDims(); - int[][] newDims = new int[convertedDims.size()][]; + Object[] dims = input.getDims(); + Object[] newDims = new Object[convertedDims.size()]; for (int i = 0; i < convertedDims.size(); ++i) { - IntBuffer converter = converterArray[i]; if (i >= dims.length) { continue; } - - if (dims[i] == null && convertMissingDimsFlags.get(i)) { - newDims[i] = EMPTY_STR_DIM; - continue; - } - - if (converter == null) { - newDims[i] = dims[i]; - continue; - } - - newDims[i] = new int[dims[i].length]; - - for (int j = 0; j < dims[i].length; ++j) { - newDims[i][j] = converter.get(dims[i][j]); - } + newDims[i] = mergers.get(i).convertSegmentRowValuesToMergedRowValues(dims[i], indexNumber); } final Rowboat retVal = new Rowboat( input.getTimestamp(), newDims, input.getMetrics(), - input.getRowNum() + input.getRowNum(), + input.getHandlers() ); retVal.addRow(indexNumber, input.getRowNum()); @@ -1533,7 +1261,8 @@ public Rowboat apply(Rowboat lhs, Rowboat rhs) lhs.getTimestamp(), lhs.getDims(), metrics, - lhs.getRowNum() + lhs.getRowNum(), + lhs.getHandlers() ); for (Rowboat rowboat : Arrays.asList(lhs, rhs)) { diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index a255bb559bfb..b504a6db1993 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -42,7 +41,6 @@ import com.metamx.common.io.smoosh.FileSmoosher; import com.metamx.common.io.smoosh.SmooshedWriter; import com.metamx.common.logger.Logger; -import io.druid.collections.CombiningIterable; import io.druid.common.utils.JodaUtils; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.column.Column; @@ -50,26 +48,15 @@ import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; -import io.druid.segment.data.BitmapSerdeFactory; -import io.druid.segment.data.ByteBufferWriter; import io.druid.segment.data.CompressedObjectStrategy; -import io.druid.segment.data.CompressedVSizeIndexedV3Writer; -import io.druid.segment.data.CompressedVSizeIntsIndexedWriter; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.IOPeon; -import io.druid.segment.data.Indexed; -import io.druid.segment.data.IndexedIntsWriter; -import io.druid.segment.data.IndexedRTree; import io.druid.segment.data.TmpFileIOPeon; -import io.druid.segment.data.VSizeIndexedIntsWriter; -import io.druid.segment.data.VSizeIndexedWriter; import io.druid.segment.serde.ComplexColumnPartSerde; import io.druid.segment.serde.ComplexColumnSerializer; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; -import io.druid.segment.serde.DictionaryEncodedColumnPartSerde; import io.druid.segment.serde.FloatGenericColumnPartSerde; import io.druid.segment.serde.LongGenericColumnPartSerde; import org.apache.commons.io.FileUtils; @@ -79,11 +66,9 @@ import java.io.ByteArrayOutputStream; import java.io.Closeable; import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.IntBuffer; -import java.nio.MappedByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -184,18 +169,20 @@ public void close() throws IOException final List dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size()); mergeCapabilities(adapters, mergedDimensions, metricsValueTypes, metricTypeNames, dimCapabilities); + final DimensionHandler[] handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities); + final List mergers = new ArrayList<>(); + for (int i = 0; i < mergedDimensions.size(); i++) { + mergers.add(handlers[i].makeMerger(indexSpec, v9TmpDir, ioPeon, dimCapabilities.get(i), progress)); + } + /************* Setup Dim Conversions **************/ progress.progress(); startTime = System.currentTimeMillis(); - final Map dimCardinalities = Maps.newHashMap(); - final ArrayList> dimValueWriters = setupDimValueWriters(ioPeon, mergedDimensions); final ArrayList> dimConversions = Lists.newArrayListWithCapacity(adapters.size()); final ArrayList dimensionSkipFlag = Lists.newArrayListWithCapacity(mergedDimensions.size()); - final ArrayList dimHasNullFlags = Lists.newArrayListWithCapacity(mergedDimensions.size()); final ArrayList convertMissingDimsFlags = Lists.newArrayListWithCapacity(mergedDimensions.size()); writeDimValueAndSetupDimConversion( - adapters, progress, mergedDimensions, dimCardinalities, dimValueWriters, dimensionSkipFlag, dimConversions, - convertMissingDimsFlags, dimHasNullFlags + adapters, progress, mergedDimensions, mergers ); log.info("Completed dim conversions in %,d millis.", System.currentTimeMillis() - startTime); @@ -205,53 +192,43 @@ public void close() throws IOException adapters, mergedDimensions, mergedMetrics, - dimConversions, - convertMissingDimsFlags, - rowMergerFn + rowMergerFn, + dimCapabilities, + handlers, + mergers ); final LongColumnSerializer timeWriter = setupTimeWriter(ioPeon, indexSpec); - final ArrayList dimWriters = setupDimensionWriters( - ioPeon, mergedDimensions, dimCapabilities, dimCardinalities, indexSpec - ); final ArrayList metWriters = setupMetricsWriters( ioPeon, mergedMetrics, metricsValueTypes, metricTypeNames, indexSpec ); final List rowNumConversions = Lists.newArrayListWithCapacity(adapters.size()); - final ArrayList nullRowsList = Lists.newArrayListWithCapacity(mergedDimensions.size()); - for (int i = 0; i < mergedDimensions.size(); ++i) { - nullRowsList.add(indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap()); - } - mergeIndexesAndWriteColumns( - adapters, progress, theRows, timeWriter, dimWriters, metWriters, - dimensionSkipFlag, rowNumConversions, nullRowsList, dimHasNullFlags - ); - /************ Create Inverted Indexes *************/ - progress.progress(); - final ArrayList> bitmapIndexWriters = setupBitmapIndexWriters( - ioPeon, mergedDimensions, indexSpec - ); - final ArrayList> spatialIndexWriters = setupSpatialIndexWriters( - ioPeon, mergedDimensions, indexSpec, dimCapabilities - ); - makeInvertedIndexes( - adapters, progress, mergedDimensions, indexSpec, v9TmpDir, rowNumConversions, - nullRowsList, dimValueWriters, bitmapIndexWriters, spatialIndexWriters, dimConversions + mergeIndexesAndWriteColumns( + adapters, progress, theRows, timeWriter, metWriters, rowNumConversions, mergers ); - /************ Finalize Build Columns *************/ - progress.progress(); + /************ Create Inverted Indexes and Finalize Build Columns *************/ + final String section = "build inverted index and columns"; + progress.startSection(section); makeTimeColumn(v9Smoosher, progress, timeWriter); makeMetricsColumns(v9Smoosher, progress, mergedMetrics, metricsValueTypes, metricTypeNames, metWriters); - makeDimensionColumns( - v9Smoosher, progress, indexSpec, mergedDimensions, dimensionSkipFlag, dimCapabilities, - dimValueWriters, dimWriters, bitmapIndexWriters, spatialIndexWriters - ); + + for(int i = 0; i < mergedDimensions.size(); i++) { + DimensionMergerV9 merger = (DimensionMergerV9) mergers.get(i); + merger.writeIndexes(rowNumConversions, closer); + if (merger.canSkip()) { + continue; + } + ColumnDescriptor columnDesc = merger.makeColumnDescriptor(); + makeColumn(v9Smoosher, mergedDimensions.get(i), columnDesc); + } + + progress.stopSection(section); /************* Make index.drd & metadata.drd files **************/ progress.progress(); makeIndexBinary( - v9Smoosher, adapters, outDir, mergedDimensions, dimensionSkipFlag, mergedMetrics, progress, indexSpec + v9Smoosher, adapters, outDir, mergedDimensions, mergedMetrics, progress, indexSpec, mergers ); makeMetadataBinary(v9Smoosher, progress, segmentMetadata); @@ -286,10 +263,10 @@ private void makeIndexBinary( final List adapters, final File outDir, final List mergedDimensions, - final ArrayList dimensionSkipFlag, final List mergedMetrics, final ProgressIndicator progress, - final IndexSpec indexSpec + final IndexSpec indexSpec, + final List mergers ) throws IOException { final String section = "make index.drd"; @@ -300,7 +277,7 @@ private void makeIndexBinary( final Set finalColumns = Sets.newLinkedHashSet(); finalColumns.addAll(mergedMetrics); for (int i = 0; i < mergedDimensions.size(); ++i) { - if (dimensionSkipFlag.get(i)) { + if (mergers.get(i).canSkip()) { continue; } finalColumns.add(mergedDimensions.get(i)); @@ -343,68 +320,6 @@ private void makeIndexBinary( progress.stopSection(section); } - private void makeDimensionColumns( - final FileSmoosher v9Smoosher, - final ProgressIndicator progress, - final IndexSpec indexSpec, - final List mergedDimensions, - final ArrayList dimensionSkipFlag, - final List dimCapabilities, - final ArrayList> dimValueWriters, - final ArrayList dimWriters, - final ArrayList> bitmapIndexWriters, - final ArrayList> spatialIndexWriters - ) throws IOException - { - final String section = "make dimension columns"; - progress.startSection(section); - - long startTime = System.currentTimeMillis(); - final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); - final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); - for (int i = 0; i < mergedDimensions.size(); ++i) { - long dimStartTime = System.currentTimeMillis(); - final String dim = mergedDimensions.get(i); - final IndexedIntsWriter dimWriter = dimWriters.get(i); - final GenericIndexedWriter bitmapIndexWriter = bitmapIndexWriters.get(i); - final ByteBufferWriter spatialIndexWriter = spatialIndexWriters.get(i); - - dimWriter.close(); - bitmapIndexWriter.close(); - if (spatialIndexWriter != null) { - spatialIndexWriter.close(); - } - if (dimensionSkipFlag.get(i)) { - continue; - } - - boolean hasMultiValue = dimCapabilities.get(i).hasMultipleValues(); - - final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); - builder.setValueType(ValueType.STRING); - builder.setHasMultipleValues(hasMultiValue); - final DictionaryEncodedColumnPartSerde.SerializerBuilder partBuilder = DictionaryEncodedColumnPartSerde - .serializerBuilder() - .withDictionary(dimValueWriters.get(i)) - .withValue( - dimWriters.get(i), - hasMultiValue, - compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED - ) - .withBitmapSerdeFactory(bitmapSerdeFactory) - .withBitmapIndex(bitmapIndexWriters.get(i)) - .withSpatialIndex(spatialIndexWriters.get(i)) - .withByteOrder(IndexIO.BYTE_ORDER); - final ColumnDescriptor serdeficator = builder - .addSerde(partBuilder.build()) - .build(); - makeColumn(v9Smoosher, dim, serdeficator); - log.info("Completed dimension column[%s] in %,d millis.", dim, System.currentTimeMillis() - dimStartTime); - } - log.info("Completed dimension columns in %,d millis.", System.currentTimeMillis() - startTime); - progress.stopSection(section); - } - private void makeMetricsColumns( final FileSmoosher v9Smoosher, final ProgressIndicator progress, @@ -464,7 +379,6 @@ private void makeMetricsColumns( progress.stopSection(section); } - private void makeTimeColumn( final FileSmoosher v9Smoosher, final ProgressIndicator progress, @@ -514,173 +428,14 @@ private void makeColumn( } } - private void makeInvertedIndexes( - final List adapters, - final ProgressIndicator progress, - final List mergedDimensions, - final IndexSpec indexSpec, - final File v9OutDir, - final List rowNumConversions, - final ArrayList nullRowsList, - final ArrayList> dimValueWriters, - final ArrayList> bitmapIndexWriters, - final ArrayList> spatialIndexWriters, - final ArrayList> dimConversions - ) throws IOException - { - final String section = "build inverted index"; - progress.startSection(section); - - long startTime = System.currentTimeMillis(); - final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); - for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) { - String dimension = mergedDimensions.get(dimIndex); - long dimStartTime = System.currentTimeMillis(); - - // write dim values to one single file because we need to read it - File dimValueFile = IndexIO.makeDimFile(v9OutDir, dimension); - FileOutputStream fos = new FileOutputStream(dimValueFile); - ByteStreams.copy(dimValueWriters.get(dimIndex).combineStreams(), fos); - fos.close(); - - final MappedByteBuffer dimValsMapped = Files.map(dimValueFile); - try (Closeable dimValsMappedUnmapper = new Closeable() - { - @Override - public void close() - { - ByteBufferUtils.unmap(dimValsMapped); - } - }) { - Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY); - - ByteBufferWriter spatialIndexWriter = spatialIndexWriters.get(dimIndex); - RTree tree = null; - if (spatialIndexWriter != null) { - BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory(); - tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory); - } - - IndexSeeker[] dictIdSeeker = toIndexSeekers(adapters, dimConversions, dimension); - - ImmutableBitmap nullRowBitmap = bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap( - nullRowsList.get(dimIndex) - ); - - //Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result. - for (int dictId = 0; dictId < dimVals.size(); dictId++) { - progress.progress(); - List> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size()); - for (int j = 0; j < adapters.size(); ++j) { - int seekedDictId = dictIdSeeker[j].seek(dictId); - if (seekedDictId != IndexSeeker.NOT_EXIST) { - convertedInverteds.add( - new ConvertingIndexedInts( - adapters.get(j).getBitmapIndex(dimension, seekedDictId), rowNumConversions.get(j) - ) - ); - } - } - - MutableBitmap bitset = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap(); - for (Integer row : CombiningIterable.createSplatted( - convertedInverteds, - Ordering.natural().nullsFirst() - )) { - if (row != INVALID_ROW) { - bitset.add(row); - } - } - - ImmutableBitmap bitmapToWrite = bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(bitset); - if ((dictId == 0) && (Iterables.getFirst(dimVals, "") == null)) { - bitmapToWrite = nullRowBitmap.union(bitmapToWrite); - } - bitmapIndexWriters.get(dimIndex).write(bitmapToWrite); - - if (spatialIndexWriter != null) { - String dimVal = dimVals.get(dictId); - if (dimVal != null) { - List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); - float[] coords = new float[stringCoords.size()]; - for (int j = 0; j < coords.length; j++) { - coords[j] = Float.valueOf(stringCoords.get(j)); - } - tree.insert(coords, bitset); - } - } - } - if (spatialIndexWriter != null) { - spatialIndexWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); - } - log.info( - "Completed dim[%s] inverted with cardinality[%,d] in %,d millis.", - dimension, - dimVals.size(), - System.currentTimeMillis() - dimStartTime - ); - } - } - log.info("Completed inverted index in %,d millis.", System.currentTimeMillis() - startTime); - progress.stopSection(section); - } - - - private ArrayList> setupBitmapIndexWriters( - final IOPeon ioPeon, - final List mergedDimensions, - final IndexSpec indexSpec - ) throws IOException - { - ArrayList> writers = Lists.newArrayListWithCapacity(mergedDimensions.size()); - final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); - for (String dimension : mergedDimensions) { - GenericIndexedWriter writer = new GenericIndexedWriter<>( - ioPeon, String.format("%s.inverted", dimension), bitmapSerdeFactory.getObjectStrategy() - ); - writer.open(); - writers.add(writer); - } - return writers; - } - - private ArrayList> setupSpatialIndexWriters( - final IOPeon ioPeon, - final List mergedDimensions, - final IndexSpec indexSpec, - final List dimCapabilities - ) throws IOException - { - ArrayList> writers = Lists.newArrayListWithCapacity(mergedDimensions.size()); - final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); - for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) { - if (dimCapabilities.get(dimIndex).hasSpatialIndexes()) { - BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory(); - ByteBufferWriter writer = new ByteBufferWriter<>( - ioPeon, - String.format("%s.spatial", mergedDimensions.get(dimIndex)), - new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapFactory) - ); - writer.open(); - writers.add(writer); - } else { - writers.add(null); - } - } - return writers; - } - private void mergeIndexesAndWriteColumns( final List adapters, final ProgressIndicator progress, final Iterable theRows, final LongColumnSerializer timeWriter, - final ArrayList dimWriters, final ArrayList metWriters, - final ArrayList dimensionSkipFlag, final List rowNumConversions, - final ArrayList nullRowsList, - final ArrayList dimHasNullFlags + final List mergers ) throws IOException { final String section = "walk through and merge rows"; @@ -704,19 +459,13 @@ private void mergeIndexesAndWriteColumns( metWriters.get(i).serialize(metrics[i]); } - int[][] dims = theRow.getDims(); + Object[] dims = theRow.getDims(); for (int i = 0; i < dims.length; ++i) { - if (dimensionSkipFlag.get(i)) { + DimensionMerger merger = mergers.get(i); + if (merger.canSkip()) { continue; } - if (dims[i] == null || dims[i].length == 0) { - nullRowsList.get(i).add(rowCount); - } else if (dimHasNullFlags.get(i) && dims[i].length == 1 && dims[i][0] == 0) { - // 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 nullRowsList. - nullRowsList.get(i).add(rowCount); - } - dimWriters.get(i).add(dims[i]); + merger.processMergedRow(dims[i]); } for (Map.Entry> comprisedRow : theRow.getComprisedRows().entrySet()) { @@ -791,153 +540,18 @@ private ArrayList setupMetricsWriters( return metWriters; } - private ArrayList setupDimensionWriters( - final IOPeon ioPeon, - final List mergedDimensions, - final List dimCapabilities, - final Map dimCardinalities, - final IndexSpec indexSpec - ) throws IOException - { - ArrayList dimWriters = Lists.newArrayListWithCapacity(mergedDimensions.size()); - final CompressedObjectStrategy.CompressionStrategy dimCompression = indexSpec.getDimensionCompression(); - for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) { - String dim = mergedDimensions.get(dimIndex); - int cardinality = dimCardinalities.get(dim); - ColumnCapabilitiesImpl capabilities = dimCapabilities.get(dimIndex); - String filenameBase = String.format("%s.forward_dim", dim); - IndexedIntsWriter writer; - if (capabilities.hasMultipleValues()) { - writer = (dimCompression != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) - ? CompressedVSizeIndexedV3Writer.create(ioPeon, filenameBase, cardinality, dimCompression) - : new VSizeIndexedWriter(ioPeon, filenameBase, cardinality); - } else { - writer = (dimCompression != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) - ? CompressedVSizeIntsIndexedWriter.create(ioPeon, filenameBase, cardinality, dimCompression) - : new VSizeIndexedIntsWriter(ioPeon, filenameBase, cardinality); - } - writer.open(); - // we will close these writers in another method after we added all the values - dimWriters.add(writer); - } - return dimWriters; - } - - private ArrayList> setupDimValueWriters( - final IOPeon ioPeon, - final List mergedDimensions - ) - throws IOException - { - ArrayList> dimValueWriters = Lists.newArrayListWithCapacity(mergedDimensions.size()); - for (String dimension : mergedDimensions) { - final GenericIndexedWriter writer = new GenericIndexedWriter<>( - ioPeon, String.format("%s.dim_values", dimension), GenericIndexed.STRING_STRATEGY - ); - writer.open(); - dimValueWriters.add(writer); - } - return dimValueWriters; - } - private void writeDimValueAndSetupDimConversion( final List indexes, final ProgressIndicator progress, final List mergedDimensions, - final Map dimensionCardinalities, - final ArrayList> dimValueWriters, - final ArrayList dimensionSkipFlag, - final List> dimConversions, - final ArrayList convertMissingDimsFlags, - final ArrayList dimHasNullFlags + final List mergers ) throws IOException { final String section = "setup dimension conversions"; progress.startSection(section); - for (int i = 0; i < indexes.size(); ++i) { - dimConversions.add(Maps.newHashMap()); - } - for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) { - long dimStartTime = System.currentTimeMillis(); - String dimension = mergedDimensions.get(dimIndex); - boolean dimHasNull = false; - boolean dimHasValues = false; - boolean dimAbsentFromSomeIndex = false; - - int numMergeIndex = 0; - Indexed dimValueLookup = null; - Indexed[] dimValueLookups = new Indexed[indexes.size() + 1]; - for (int i = 0; i < indexes.size(); i++) { - Indexed dimValues = indexes.get(i).getDimValueLookup(dimension); - if (!isNullColumn(dimValues)) { - dimHasValues = true; - dimHasNull |= dimValues.indexOf(null) >= 0; - dimValueLookups[i] = dimValueLookup = dimValues; - numMergeIndex++; - } else { - dimAbsentFromSomeIndex = true; - } - } - - boolean convertMissingDims = dimHasValues && dimAbsentFromSomeIndex; - convertMissingDimsFlags.add(convertMissingDims); - - /* - * 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 - * 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. - */ - if (convertMissingDims && !dimHasNull) { - dimHasNull = true; - dimValueLookups[indexes.size()] = dimValueLookup = EMPTY_STR_DIM_VAL; - numMergeIndex++; - } - - GenericIndexedWriter writer = dimValueWriters.get(dimIndex); - - int cardinality = 0; - if (numMergeIndex > 1) { - DictionaryMergeIterator iterator = new DictionaryMergeIterator(dimValueLookups, true); - - while (iterator.hasNext()) { - writer.write(iterator.next()); - } - - for (int i = 0; i < indexes.size(); i++) { - if (dimValueLookups[i] != null && iterator.needConversion(i)) { - dimConversions.get(i).put(dimension, iterator.conversions[i]); - } - } - cardinality = iterator.counter; - } else if (numMergeIndex == 1) { - for (String value : dimValueLookup) { - writer.write(value); - } - cardinality = dimValueLookup.size(); - } - - // Mark if this dim has the null/empty str value in its dictionary, used for determining nullRowsList later. - dimHasNullFlags.add(dimHasNull); - - log.info( - "Completed dim[%s] conversions with cardinality[%,d] in %,d millis.", - dimension, - cardinality, - System.currentTimeMillis() - dimStartTime - ); - dimensionCardinalities.put(dimension, cardinality); - writer.close(); - - if (cardinality == 0) { - log.info(String.format("Skipping [%s], it is empty!", dimension)); - dimensionSkipFlag.add(true); - continue; - } - dimensionSkipFlag.add(false); + mergers.get(dimIndex).writeMergedValueMetadata(indexes); } progress.stopSection(section); } @@ -956,7 +570,7 @@ private void mergeCapabilities( ColumnCapabilitiesImpl mergedCapabilities = capabilitiesMap.get(dimension); if (mergedCapabilities == null) { mergedCapabilities = new ColumnCapabilitiesImpl(); - mergedCapabilities.setType(ValueType.STRING); + mergedCapabilities.setType(null); } capabilitiesMap.put(dimension, mergedCapabilities.merge(adapter.getCapabilities(dimension))); } diff --git a/processing/src/main/java/io/druid/segment/IndexableAdapter.java b/processing/src/main/java/io/druid/segment/IndexableAdapter.java index e3aaed1e7e37..cc34913d8525 100644 --- a/processing/src/main/java/io/druid/segment/IndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/IndexableAdapter.java @@ -24,6 +24,8 @@ import io.druid.segment.data.IndexedInts; import org.joda.time.Interval; +import java.util.Map; + /** * An adapter to an index */ @@ -37,7 +39,7 @@ public interface IndexableAdapter Indexed getMetricNames(); - Indexed getDimValueLookup(String dimension); + Indexed getDimValueLookup(String dimension); Iterable getRows(); @@ -48,4 +50,6 @@ public interface IndexableAdapter ColumnCapabilities getCapabilities(String column); Metadata getMetadata(); + + Map getDimensionHandlers(); } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndex.java b/processing/src/main/java/io/druid/segment/QueryableIndex.java index 3daa70896d8f..d2a2c516cae0 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndex.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndex.java @@ -25,6 +25,7 @@ import java.io.Closeable; import java.io.IOException; +import java.util.Map; /** */ @@ -36,6 +37,7 @@ public interface QueryableIndex extends ColumnSelector, Closeable public Indexed getAvailableDimensions(); public BitmapFactory getBitmapFactoryForDimensions(); public Metadata getMetadata(); + public Map getDimensionHandlers(); /** * The close method shouldn't actually be here as this is nasty. We will adjust it in the future. diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index a102eae92fd2..a33e64f5f673 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -36,7 +36,6 @@ import io.druid.segment.column.IndexedFloatsGenericColumn; import io.druid.segment.column.IndexedLongsGenericColumn; import io.druid.segment.column.ValueType; -import io.druid.segment.data.ArrayBasedIndexedInts; import io.druid.segment.data.BitmapCompressedIndexedInts; import io.druid.segment.data.EmptyIndexedInts; import io.druid.segment.data.Indexed; @@ -46,9 +45,11 @@ import org.joda.time.Interval; import java.io.Closeable; +import java.util.Collection; import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.NoSuchElementException; import java.util.Set; @@ -76,10 +77,11 @@ public QueryableIndexIndexableAdapter(QueryableIndex input) if (col == null) { log.warn("Wtf!? column[%s] didn't exist!?!?!?", dim); - } else if (col.getDictionaryEncoding() != null) { - availableDimensions.add(dim); } else { - log.info("No dictionary on dimension[%s]", dim); + if (col.getDictionaryEncoding() == null) { + log.info("No dictionary on dimension[%s]", dim); + } + availableDimensions.add(dim); } } @@ -117,7 +119,7 @@ public Indexed getMetricNames() } @Override - public Indexed getDimValueLookup(String dimension) + public Indexed getDimValueLookup(String dimension) { final Column column = input.getColumn(dimension); @@ -131,12 +133,12 @@ public Indexed getDimValueLookup(String dimension) return null; } - return new Indexed() + return new Indexed() { @Override - public Class getClazz() + public Class getClazz() { - return String.class; + return Comparable.class; } @Override @@ -146,19 +148,19 @@ public int size() } @Override - public String get(int index) + public Comparable get(int index) { return dict.lookupName(index); } @Override - public int indexOf(String value) + public int indexOf(Comparable value) { return dict.lookupId(value); } @Override - public Iterator iterator() + public Iterator iterator() { return IndexedIterable.create(this).iterator(); } @@ -177,28 +179,31 @@ public Iterator iterator() { final GenericColumn timestamps = input.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn(); final Object[] metrics; - - final DictionaryEncodedColumn[] dictionaryEncodedColumns; + final Closeable[] columns; final int numMetrics = getMetricNames().size(); + final DimensionHandler[] handlers = new DimensionHandler[availableDimensions.size()]; + Collection handlerSet = input.getDimensionHandlers().values(); + int currRow = 0; boolean done = false; { - this.dictionaryEncodedColumns = FluentIterable - .from(getDimensionNames()) + handlerSet.toArray(handlers); + this.columns = FluentIterable + .from(handlerSet) .transform( - new Function() + new Function() { @Override - public DictionaryEncodedColumn apply(String dimName) + public Closeable apply(DimensionHandler handler) { - return input.getColumn(dimName) - .getDictionaryEncoding(); + Column column = input.getColumn(handler.getDimensionName()); + return handler.getSubColumn(column); } } - ).toArray(DictionaryEncodedColumn.class); + ).toArray(Closeable.class); final Indexed availableMetrics = getMetricNames(); metrics = new Object[availableMetrics.size()]; @@ -230,10 +235,8 @@ public boolean hasNext() CloseQuietly.close((Closeable) metric); } } - for (Object dimension : dictionaryEncodedColumns) { - if (dimension instanceof Closeable) { - CloseQuietly.close((Closeable) dimension); - } + for (Closeable dimension : columns) { + CloseQuietly.close(dimension); } done = true; } @@ -247,22 +250,11 @@ public Rowboat next() throw new NoSuchElementException(); } - final int[][] dims = new int[dictionaryEncodedColumns.length][]; + final Object[] dims = new Object[columns.length]; int dimIndex = 0; - for (final DictionaryEncodedColumn dict : dictionaryEncodedColumns) { - final IndexedInts dimVals; - if (dict.hasMultipleValues()) { - dimVals = dict.getMultiValueRow(currRow); - } else { - dimVals = new ArrayBasedIndexedInts(new int[]{dict.getSingleValueRow(currRow)}); - } - - int[] theVals = new int[dimVals.size()]; - for (int j = 0; j < theVals.length; ++j) { - theVals[j] = dimVals.get(j); - } - - dims[dimIndex++] = theVals; + for (final Closeable column : columns) { + dims[dimIndex] = handlers[dimIndex].getRowValueArrayFromColumn(column, currRow); + dimIndex++; } Object[] metricArray = new Object[numMetrics]; @@ -277,7 +269,7 @@ public Rowboat next() } final Rowboat retVal = new Rowboat( - timestamps.getLongSingleValueRow(currRow), dims, metricArray, currRow + timestamps.getLongSingleValueRow(currRow), dims, metricArray, currRow, handlers ); ++currRow; @@ -361,4 +353,10 @@ 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 50cade3c1316..db724373a51a 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -192,6 +192,12 @@ public ColumnCapabilities getColumnCapabilities(String column) return getColumnCapabilites(index, column); } + @Override + public Map getDimensionHandlers() + { + return index.getDimensionHandlers(); + } + @Override public String getColumnTypeName(String columnName) { @@ -462,13 +468,13 @@ private DimensionSelector makeDimensionSelectorUndecorated( ); } - DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimension); + DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimension); if (cachedColumn == null) { cachedColumn = columnDesc.getDictionaryEncoding(); dictionaryColumnCache.put(dimension, cachedColumn); } - final DictionaryEncodedColumn column = cachedColumn; + final DictionaryEncodedColumn column = cachedColumn; if (column == null) { return NULL_DIMENSION_SELECTOR; @@ -739,7 +745,7 @@ public String get() } if (cachedColumnVals instanceof DictionaryEncodedColumn) { - final DictionaryEncodedColumn columnVals = (DictionaryEncodedColumn) cachedColumnVals; + final DictionaryEncodedColumn columnVals = (DictionaryEncodedColumn) cachedColumnVals; if (columnVals.hasMultipleValues()) { return new ObjectColumnSelector() { diff --git a/processing/src/main/java/io/druid/segment/Rowboat.java b/processing/src/main/java/io/druid/segment/Rowboat.java index bfe6049250ab..cded51f5f028 100644 --- a/processing/src/main/java/io/druid/segment/Rowboat.java +++ b/processing/src/main/java/io/druid/segment/Rowboat.java @@ -32,22 +32,25 @@ public class Rowboat implements Comparable { private final long timestamp; - private final int[][] dims; + private final Object[] dims; private final Object[] metrics; private final int rowNum; private final Map> comprisedRows; + private final DimensionHandler[] handlers; public Rowboat( long timestamp, - int[][] dims, + Object[] dims, Object[] metrics, - int rowNum + int rowNum, + DimensionHandler[] handlers ) { this.timestamp = timestamp; this.dims = dims; this.metrics = metrics; this.rowNum = rowNum; + this.handlers = handlers; this.comprisedRows = Maps.newHashMap(); } @@ -57,7 +60,7 @@ public long getTimestamp() return timestamp; } - public int[][] getDims() + public Object[] getDims() { return dims; } @@ -82,6 +85,11 @@ public Map> getComprisedRows() return comprisedRows; } + public DimensionHandler[] getHandlers() + { + return handlers; + } + public int getRowNum() { return rowNum; @@ -98,8 +106,8 @@ public int compareTo(Rowboat rhs) int index = 0; while (retVal == 0 && index < dims.length) { - int[] lhsVals = dims[index]; - int[] rhsVals = rhs.dims[index]; + Object lhsVals = dims[index]; + Object rhsVals = rhs.dims[index]; if (lhsVals == null) { if (rhsVals == null) { @@ -113,13 +121,8 @@ public int compareTo(Rowboat rhs) return 1; } - retVal = Ints.compare(lhsVals.length, rhsVals.length); - - int valsIndex = 0; - while (retVal == 0 && valsIndex < lhsVals.length) { - retVal = Ints.compare(lhsVals[valsIndex], rhsVals[valsIndex]); - ++valsIndex; - } + DimensionHandler handler = handlers[index]; + retVal = handler.compareSortedEncodedArrays(lhsVals, rhsVals); ++index; } diff --git a/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java b/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java index b4dcc742de13..e9976bf6fde1 100644 --- a/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java @@ -26,6 +26,8 @@ import io.druid.segment.data.IndexedInts; import org.joda.time.Interval; +import java.util.Map; + /** */ public class RowboatFilteringIndexAdapter implements IndexableAdapter @@ -64,7 +66,7 @@ public Indexed getMetricNames() } @Override - public Indexed getDimValueLookup(String dimension) + public Indexed getDimValueLookup(String dimension) { return baseAdapter.getDimValueLookup(dimension); } @@ -98,4 +100,10 @@ public Metadata getMetadata() { return baseAdapter.getMetadata(); } + + @Override + public Map getDimensionHandlers() + { + return baseAdapter.getDimensionHandlers(); + } } diff --git a/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java b/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java index 5d6aa0452c34..eb253f35e893 100644 --- a/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java +++ b/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java @@ -20,9 +20,11 @@ package io.druid.segment; import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; import com.metamx.collections.bitmap.BitmapFactory; import com.metamx.common.io.smoosh.SmooshedFileMapper; import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.data.Indexed; import org.joda.time.Interval; @@ -40,6 +42,7 @@ public class SimpleQueryableIndex implements QueryableIndex private final Map columns; private final SmooshedFileMapper fileMapper; private final Metadata metadata; + private final Map dimensionHandlers; public SimpleQueryableIndex( Interval dataInterval, @@ -59,6 +62,8 @@ public SimpleQueryableIndex( this.columns = columns; this.fileMapper = fileMapper; this.metadata = metadata; + this.dimensionHandlers = Maps.newLinkedHashMap(); + initDimensionHandlers(); } @Override @@ -108,4 +113,19 @@ public Metadata getMetadata() { return metadata; } + + @Override + public Map getDimensionHandlers() + { + return dimensionHandlers; + } + + private void initDimensionHandlers() + { + for (String dim : availableDimensions) { + ColumnCapabilities capabilities = getColumn(dim).getCapabilities(); + DimensionHandler handler = DimensionHandlerUtil.getHandlerFromCapabilities(dim, capabilities); + dimensionHandlers.put(dim, handler); + } + } } diff --git a/processing/src/main/java/io/druid/segment/StorageAdapter.java b/processing/src/main/java/io/druid/segment/StorageAdapter.java index 418a45e4ac10..603c8dde7413 100644 --- a/processing/src/main/java/io/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/StorageAdapter.java @@ -24,6 +24,8 @@ import org.joda.time.DateTime; import org.joda.time.Interval; +import java.util.Map; + /** */ public interface StorageAdapter extends CursorFactory @@ -48,6 +50,7 @@ public interface StorageAdapter extends CursorFactory public Comparable getMaxValue(String column); public Capabilities getCapabilities(); public ColumnCapabilities getColumnCapabilities(String column); + public Map getDimensionHandlers(); /** * Like {@link ColumnCapabilities#getType()}, but may return a more descriptive string for complex columns. diff --git a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java new file mode 100644 index 000000000000..f13573f21b14 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java @@ -0,0 +1,267 @@ +/* + * 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.Function; +import com.google.common.primitives.Ints; +import com.metamx.common.logger.Logger; +import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.DictionaryEncodedColumn; +import io.druid.segment.data.IOPeon; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; + +import java.io.Closeable; +import java.io.File; +import java.lang.reflect.Array; +import java.util.Arrays; +import java.util.Comparator; + +public class StringDimensionHandler implements DimensionHandler +{ + private static final Logger log = new Logger(StringDimensionHandler.class); + + private final String dimensionName; + + public StringDimensionHandler(String dimensionName) + { + this.dimensionName = dimensionName; + } + + @Override + public String getDimensionName() + { + return dimensionName; + } + + @Override + public int getLengthFromEncodedArray(int[] dimVals) + { + return dimVals.length; + } + + @Override + public int compareSortedEncodedArrays(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; + } + + @Override + public void validateSortedEncodedArrays( + int[] lhs, + int[] rhs, + Indexed lhsEncodings, + Indexed rhsEncodings + ) throws SegmentValidationException + { + if (lhs == null || rhs == null) { + if (lhs != rhs) { + 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]; + + if (dIdex1 == dIdex2) { + continue; + } + + 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) + { + return column.getDictionaryEncoding(); + } + + @Override + public Object getRowValueArrayFromColumn(Closeable column, int currRow) + { + 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; + } + + @Override + public DimensionIndexer makeIndexer() + { + return new StringDimensionIndexer(); + } + + @Override + public DimensionMergerV9 makeMerger( + IndexSpec indexSpec, + File outDir, + IOPeon ioPeon, + ColumnCapabilities capabilities, + ProgressIndicator progress + ) + { + return new StringDimensionMergerV9(dimensionName, indexSpec, outDir, ioPeon, capabilities, progress); + } + + @Override + public DimensionMergerLegacy makeLegacyMerger( + IndexSpec indexSpec, + File outDir, + IOPeon ioPeon, + ColumnCapabilities capabilities, + ProgressIndicator progress + ) + { + return new StringDimensionMergerLegacy(dimensionName, indexSpec, outDir, ioPeon, capabilities, progress); + } + + public static final Function STRING_TRANSFORMER = new Function() + { + @Override + public String apply(final Object o) + { + if (o == null) { + return null; + } + if (o instanceof String) { + return (String) o; + } + return o.toString(); + } + }; + + public static final Comparator ENCODED_COMPARATOR = new Comparator() + { + @Override + public int compare(Integer o1, Integer o2) + { + if (o1 == null) { + return o2 == null ? 0 : -1; + } + if (o2 == null) { + return 1; + } + return o1.compareTo(o2); + } + }; + + public static final Comparator UNENCODED_COMPARATOR = new Comparator() + { + @Override + public int compare(String o1, String o2) + { + if (o1 == null) { + return o2 == null ? 0 : -1; + } + if (o2 == null) { + return 1; + } + return o1.compareTo(o2); + } + }; +} diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java new file mode 100644 index 000000000000..8aa2532d187a --- /dev/null +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -0,0 +1,588 @@ +/* + * 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.Predicate; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.primitives.Ints; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.MutableBitmap; +import com.metamx.common.logger.Logger; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.extraction.ExtractionFn; +import io.druid.query.filter.DruidPredicateFactory; +import io.druid.query.filter.ValueMatcher; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedIterable; +import io.druid.segment.filter.BooleanValueMatcher; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexStorageAdapter; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public class StringDimensionIndexer implements DimensionIndexer +{ + private static final Logger log = new Logger(StringDimensionIndexer.class); + + private static class DimensionDictionary + { + private String minValue = null; + private String maxValue = null; + + private final Map valueToId = Maps.newHashMap(); + + private final List idToValue = Lists.newArrayList(); + private final Object lock; + + public DimensionDictionary() + { + this.lock = new Object(); + } + + public int getId(String value) + { + synchronized (lock) { + final Integer id = valueToId.get(Strings.nullToEmpty(value)); + return id == null ? -1 : id; + } + } + + public String getValue(int id) + { + synchronized (lock) { + return Strings.emptyToNull(idToValue.get(id)); + } + } + + public boolean contains(String value) + { + synchronized (lock) { + return valueToId.containsKey(value); + } + } + + public int size() + { + synchronized (lock) { + return valueToId.size(); + } + } + + public int add(String originalValue) + { + String value = Strings.nullToEmpty(originalValue); + synchronized (lock) { + Integer prev = valueToId.get(value); + if (prev != null) { + return prev; + } + final int index = size(); + valueToId.put(value, index); + idToValue.add(value); + minValue = minValue == null || minValue.compareTo(value) > 0 ? value : minValue; + maxValue = maxValue == null || maxValue.compareTo(value) < 0 ? value : maxValue; + return index; + } + } + + public String getMinValue() + { + return minValue; + } + + public String getMaxValue() + { + return maxValue; + } + + public SortedDimensionDictionary sort() + { + synchronized (lock) { + return new SortedDimensionDictionary(idToValue, size()); + } + } + } + + private static class SortedDimensionDictionary + { + private final List sortedVals; + private final int[] idToIndex; + private final int[] indexToId; + + public SortedDimensionDictionary(List idToValue, int length) + { + Map sortedMap = Maps.newTreeMap(); + for (int id = 0; id < length; id++) { + sortedMap.put(idToValue.get(id), id); + } + this.sortedVals = Lists.newArrayList(sortedMap.keySet()); + this.idToIndex = new int[length]; + this.indexToId = new int[length]; + int index = 0; + for (Integer id : sortedMap.values()) { + idToIndex[id] = index; + indexToId[index] = id; + index++; + } + } + + public int size() + { + return sortedVals.size(); + } + + public int getUnsortedIdFromSortedId(int index) + { + return indexToId[index]; + } + + public int getSortedIdFromUnsortedId(int id) + { + return idToIndex[id]; + } + + public String getValueFromSortedId(int index) + { + return Strings.emptyToNull(sortedVals.get(index)); + } + } + + private DimensionDictionary dimLookup; + private SortedDimensionDictionary sortedLookup; + + public StringDimensionIndexer() + { + this.dimLookup = new DimensionDictionary(); + } + + @Override + public int[] processRowValsToUnsortedEncodedArray(Object dimValues) + { + final int[] encodedDimensionValues; + final int oldDictSize = dimLookup.size(); + + if (dimValues == null) { + dimLookup.add(null); + encodedDimensionValues = null; + } else if (dimValues instanceof List) { + List dimValuesList = (List) dimValues; + + // Sort multival row by their unencoded values first. + final String[] dimensionValues = new String[dimValuesList.size()]; + for (int i = 0; i < dimValuesList.size(); i++) { + dimensionValues[i] = StringDimensionHandler.STRING_TRANSFORMER.apply(dimValuesList.get(i)); + } + Arrays.sort(dimensionValues, StringDimensionHandler.UNENCODED_COMPARATOR); + + encodedDimensionValues = new int[dimensionValues.length]; + for (int i = 0; i < dimensionValues.length; i++) { + encodedDimensionValues[i] = dimLookup.add(dimensionValues[i]); + } + } else { + String transformedVal = StringDimensionHandler.STRING_TRANSFORMER.apply(dimValues); + encodedDimensionValues = new int[]{dimLookup.add(transformedVal)}; + } + + // If dictionary size has changed, the sorted lookup is no longer valid. + if (oldDictSize != dimLookup.size()) { + sortedLookup = null; + } + + return encodedDimensionValues; + } + + @Override + public Integer getSortedEncodedValueFromUnsorted(Integer unsortedIntermediateValue) + { + updateSortedLookup(); + return sortedLookup.getSortedIdFromUnsortedId(unsortedIntermediateValue); + } + + @Override + public Integer getUnsortedEncodedValueFromSorted(Integer sortedIntermediateValue) + { + updateSortedLookup(); + return sortedLookup.getUnsortedIdFromSortedId(sortedIntermediateValue); + } + + @Override + public Indexed getSortedIndexedValues() + { + updateSortedLookup(); + return new Indexed() + { + @Override + public Class getClazz() + { + return String.class; + } + + @Override + public int size() + { + return getCardinality(); + } + + @Override + public String get(int index) + { + return getActualValue(index, true); + } + + @Override + public int indexOf(String value) + { + int id = getEncodedValue(value, false); + return id < 0 ? -1 : getSortedEncodedValueFromUnsorted(id); + } + + @Override + public Iterator iterator() + { + return IndexedIterable.create(this).iterator(); + } + }; + } + + @Override + public String getMinValue() + { + return dimLookup.getMinValue(); + } + + @Override + public String getMaxValue() + { + return dimLookup.getMaxValue(); + } + + @Override + public int getCardinality() + { + return dimLookup.size(); + } + + @Override + public int compareUnsortedEncodedArrays(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) { + int lhsVal = lhs[valsIndex]; + int rhsVal = rhs[valsIndex]; + if (lhsVal != rhsVal) { + final String lhsValActual = getActualValue(lhsVal, false); + final String rhsValActual = getActualValue(rhsVal, false); + if (lhsValActual != null && rhsValActual != null) { + retVal = lhsValActual.compareTo(rhsValActual); + } else if (lhsValActual == null ^ rhsValActual == null) { + retVal = lhsValActual == null ? -1 : 1; + } + } + ++valsIndex; + } + return retVal; + } + + @Override + public boolean checkUnsortedEncodedArraysEqual(int[] lhs, int[] rhs) + { + return Arrays.equals(lhs, rhs); + } + + @Override + public int getUnsortedEncodedArrayHashCode(int[] key) + { + return Arrays.hashCode(key); + } + + @Override + public Object makeColumnValueSelector( + final DimensionSpec spec, + final IncrementalIndexStorageAdapter.EntryHolder currEntry, + final IncrementalIndex.DimensionDesc desc + ) + { + final ExtractionFn extractionFn = spec.getExtractionFn(); + + final int dimIndex = desc.getIndex(); + final int maxId = getCardinality(); + + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + final Object[] dims = currEntry.getKey().getDims(); + + int[] indices; + if (dimIndex < dims.length) { + indices = (int[]) dims[dimIndex]; + } else { + indices = null; + } + + int nullId = getEncodedValue(null, false); + List valsTmp = null; + if ((indices == null || indices.length == 0) && nullId > -1) { + if (nullId < maxId) { + valsTmp = new ArrayList<>(1); + valsTmp.add(nullId); + } + } else if (indices != null && indices.length > 0) { + valsTmp = new ArrayList<>(indices.length); + for (int i = 0; i < indices.length; i++) { + int id = indices[i]; + if (id < maxId) { + valsTmp.add(id); + } + } + } + + final List vals = valsTmp == null ? Collections.EMPTY_LIST : valsTmp; + return new IndexedInts() + { + @Override + public int size() + { + return vals.size(); + } + + @Override + public int get(int index) + { + return vals.get(index); + } + + @Override + public Iterator iterator() + { + return vals.iterator(); + } + + @Override + public void fill(int index, int[] toFill) + { + throw new UnsupportedOperationException("fill not supported"); + } + + @Override + public void close() throws IOException + { + + } + }; + } + + @Override + public int getValueCardinality() + { + return maxId; + } + + @Override + public String lookupName(int id) + { + final String strValue = getActualValue(id, false); + return extractionFn == null ? strValue : extractionFn.apply(strValue); + } + + @Override + public int lookupId(String name) + { + if (extractionFn != null) { + throw new UnsupportedOperationException( + "cannot perform lookup when applying an extraction function" + ); + } + return getEncodedValue(name, false); + } + }; + } + + @Override + public Object convertUnsortedEncodedArrayToActualArrayOrList(int[] key, boolean asList) + { + if (key == null || key.length == 0) { + return null; + } + if (key.length == 1) { + String val = getActualValue(key[0], false); + val = Strings.nullToEmpty(val); + return val; + } else { + if (asList) { + List rowVals = new ArrayList<>(key.length); + for (int i = 0; i < key.length; i++) { + String val = getActualValue(key[i], false); + rowVals.add(Strings.nullToEmpty(val)); + } + return rowVals; + } else { + String[] rowArray = new String[key.length]; + for (int i = 0; i < key.length; i++) { + String val = getActualValue(key[i], false); + rowArray[i] = Strings.nullToEmpty(val); + } + return rowArray; + } + } + } + + @Override + public int[] convertUnsortedEncodedArrayToSortedEncodedArray(int[] key) + { + 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]); + } + return sortedDimVals; + } + + @Override + public void fillBitmapsFromUnsortedEncodedArray( + int[] key, int rowNum, MutableBitmap[] bitmapIndexes, BitmapFactory factory + ) + { + for (int dimValIdx : key) { + if (bitmapIndexes[dimValIdx] == null) { + bitmapIndexes[dimValIdx] = factory.makeEmptyMutableBitmap(); + } + bitmapIndexes[dimValIdx].add(rowNum); + } + } + + @Override + public ValueMatcher makeIndexingValueMatcher( + final Comparable matchValue, + final IncrementalIndexStorageAdapter.EntryHolder holder, + final int dimIndex + ) + { + final String value = StringDimensionHandler.STRING_TRANSFORMER.apply(matchValue); + final int encodedVal = getEncodedValue(value, false); + final boolean matchOnNull = Strings.isNullOrEmpty(value); + if (encodedVal < 0 && !matchOnNull) { + return new BooleanValueMatcher(false); + } + + return new ValueMatcher() + { + @Override + public boolean matches() + { + Object[] dims = holder.getKey().getDims(); + if (dimIndex >= dims.length) { + return matchOnNull; + } + + int[] dimsInt = (int[]) dims[dimIndex]; + if (dimsInt == null || dimsInt.length == 0) { + return matchOnNull; + } + + for (int i = 0; i < dimsInt.length; i++) { + if (dimsInt[i] == encodedVal) { + return true; + } + } + return false; + } + }; + } + + @Override + public ValueMatcher makeIndexingValueMatcher( + final DruidPredicateFactory predicateFactory, + final IncrementalIndexStorageAdapter.EntryHolder holder, + final int dimIndex + ) + { + final Predicate predicate = predicateFactory.makeStringPredicate(); + final boolean matchOnNull = predicate.apply(null); + return new ValueMatcher() + { + @Override + public boolean matches() + { + Object[] dims = holder.getKey().getDims(); + if (dimIndex >= dims.length) { + return matchOnNull; + } + + int[] dimsInt = (int[]) dims[dimIndex]; + if (dimsInt == null || dimsInt.length == 0) { + return matchOnNull; + } + + for (int i = 0; i < dimsInt.length; i++) { + String finalDimVal = getActualValue(dimsInt[i], false); + if (predicate.apply(finalDimVal)) { + return true; + } + } + return false; + } + }; + } + + private void updateSortedLookup() + { + if (sortedLookup == null) { + sortedLookup = dimLookup.sort(); + } + } + + private String getActualValue(int intermediateValue, boolean idSorted) + { + if (idSorted) { + updateSortedLookup(); + return sortedLookup.getValueFromSortedId(intermediateValue); + } else { + return dimLookup.getValue(intermediateValue); + + } + } + + private int getEncodedValue(String fullValue, boolean idSorted) + { + int unsortedId = dimLookup.getId(fullValue); + + if (idSorted) { + updateSortedLookup(); + return sortedLookup.getSortedIdFromUnsortedId(unsortedId); + } else { + return unsortedId; + } + } +} diff --git a/processing/src/main/java/io/druid/segment/StringDimensionMergerLegacy.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerLegacy.java new file mode 100644 index 000000000000..b7221a8d3216 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/StringDimensionMergerLegacy.java @@ -0,0 +1,239 @@ +/* + * 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.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import com.google.common.io.ByteSink; +import com.google.common.io.ByteStreams; +import com.google.common.io.Closer; +import com.google.common.io.Files; +import com.google.common.io.OutputSupplier; +import com.google.common.primitives.Ints; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.MutableBitmap; +import com.metamx.collections.spatial.ImmutableRTree; +import com.metamx.collections.spatial.RTree; +import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy; +import com.metamx.common.ByteBufferUtils; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import io.druid.collections.CombiningIterable; +import io.druid.common.guava.FileOutputSupplier; +import io.druid.common.utils.SerializerUtils; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.data.BitmapSerdeFactory; +import io.druid.segment.data.ByteBufferWriter; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.GenericIndexedWriter; +import io.druid.segment.data.IOPeon; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedRTree; +import io.druid.segment.data.TmpFileIOPeon; +import io.druid.segment.data.VSizeIndexedWriter; + +import java.io.Closeable; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.IntBuffer; +import java.nio.MappedByteBuffer; +import java.util.List; + +public class StringDimensionMergerLegacy extends StringDimensionMergerV9 implements DimensionMergerLegacy +{ + private static final Logger log = new Logger(StringDimensionMergerLegacy.class); + + private VSizeIndexedWriter encodedValueWriterV8; + private IOPeon spatialIoPeon; + private File dictionaryFile; + + public StringDimensionMergerLegacy( + String dimensionName, + IndexSpec indexSpec, + File outDir, + IOPeon ioPeon, + ColumnCapabilities capabilities, + ProgressIndicator progress + ) + { + super(dimensionName, indexSpec, outDir, ioPeon, capabilities, progress); + } + + @Override + protected void setupEncodedValueWriter() throws IOException + { + encodedValueWriterV8 = new VSizeIndexedWriter(ioPeon, dimensionName, cardinality); + encodedValueWriterV8.open(); + } + + @Override + protected void processMergedRowHelper(int[] vals) throws IOException + { + List listToWrite = (vals == null) + ? null + : Ints.asList(vals); + encodedValueWriterV8.add(listToWrite); + } + + @Override + public void writeIndexes(List segmentRowNumConversions, Closer closer) throws IOException + { + final SerializerUtils serializerUtils = new SerializerUtils(); + long dimStartTime = System.currentTimeMillis(); + + String bmpFilename = String.format("%s.inverted", dimensionName); + bitmapWriter = new GenericIndexedWriter<>( + ioPeon, + bmpFilename, + indexSpec.getBitmapSerdeFactory().getObjectStrategy() + ); + bitmapWriter.open(); + + final MappedByteBuffer dimValsMapped = Files.map(dictionaryFile); + closer.register(new Closeable() + { + @Override + public void close() throws IOException + { + ByteBufferUtils.unmap(dimValsMapped); + } + }); + + if (!dimensionName.equals(serializerUtils.readString(dimValsMapped))) { + throw new ISE("dimensions[%s] didn't equate!? This is a major WTF moment.", dimensionName); + } + Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY); + log.info("Starting dimension[%s] with cardinality[%,d]", dimensionName, dimVals.size()); + + final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); + final BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory(); + + RTree tree = null; + spatialWriter = null; + boolean hasSpatial = capabilities.hasSpatialIndexes(); + spatialIoPeon = new TmpFileIOPeon(); + if (hasSpatial) { + BitmapFactory bmpFactory = bitmapSerdeFactory.getBitmapFactory(); + String spatialFilename = String.format("%s.spatial", dimensionName); + spatialWriter = new ByteBufferWriter( + spatialIoPeon, spatialFilename, new IndexedRTree.ImmutableRTreeObjectStrategy(bmpFactory) + ); + spatialWriter.open(); + tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory); + } + + IndexSeeker[] dictIdSeeker = toIndexSeekers(adapters, dimConversions, dimensionName); + + //Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result. + for (int dictId = 0; dictId < dimVals.size(); dictId++) { + progress.progress(); + List> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size()); + for (int j = 0; j < adapters.size(); ++j) { + int seekedDictId = dictIdSeeker[j].seek(dictId); + if (seekedDictId != IndexSeeker.NOT_EXIST) { + convertedInverteds.add( + new ConvertingIndexedInts( + adapters.get(j).getBitmapIndex(dimensionName, seekedDictId), segmentRowNumConversions.get(j) + ) + ); + } + } + + MutableBitmap bitset = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap(); + for (Integer row : CombiningIterable.createSplatted( + convertedInverteds, + Ordering.natural().nullsFirst() + )) { + if (row != IndexMerger.INVALID_ROW) { + bitset.add(row); + } + } + if ((dictId == 0) && (Iterables.getFirst(dimVals, "") == null)) { + bitset.or(nullRowsBitmap); + } + + bitmapWriter.write( + bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(bitset) + ); + + if (hasSpatial) { + String dimVal = dimVals.get(dictId); + if (dimVal != null) { + List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); + float[] coords = new float[stringCoords.size()]; + for (int j = 0; j < coords.length; j++) { + coords[j] = Float.valueOf(stringCoords.get(j)); + } + tree.insert(coords, bitset); + } + } + } + + log.info("Completed dimension[%s] in %,d millis.", dimensionName, System.currentTimeMillis() - dimStartTime); + + if (hasSpatial) { + spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); + + } + } + + @Override + public void writeValueMetadataToFile(FileOutputSupplier valueEncodingFile) throws IOException + { + final SerializerUtils serializerUtils = new SerializerUtils(); + + dictionaryWriter.close(); + serializerUtils.writeString(valueEncodingFile, dimensionName); + ByteStreams.copy(dictionaryWriter.combineStreams(), valueEncodingFile); + + // save this File reference, we will read from it later when building bitmap/spatial indexes + dictionaryFile = valueEncodingFile.getFile(); + } + + @Override + public void writeRowValuesToFile(FileOutputSupplier rowValueFile) throws IOException + { + encodedValueWriterV8.close(); + ByteStreams.copy(encodedValueWriterV8.combineStreams(), rowValueFile); + } + + @Override + public void writeIndexesToFiles( + ByteSink invertedIndexFile, + OutputSupplier spatialIndexFile + ) throws IOException + { + final SerializerUtils serializerUtils = new SerializerUtils(); + + bitmapWriter.close(); + serializerUtils.writeString(invertedIndexFile, dimensionName); + ByteStreams.copy(bitmapWriter.combineStreams(), invertedIndexFile); + + + if (capabilities.hasSpatialIndexes()) { + spatialWriter.close(); + serializerUtils.writeString(spatialIndexFile, dimensionName); + ByteStreams.copy(spatialWriter.combineStreams(), spatialIndexFile); + spatialIoPeon.cleanup(); + } + } +} \ No newline at end of file diff --git a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java new file mode 100644 index 000000000000..79cf76a196a3 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java @@ -0,0 +1,575 @@ +/* + * 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.Function; +import com.google.common.base.Splitter; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import com.google.common.io.ByteStreams; +import com.google.common.io.Closer; +import com.google.common.io.Files; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; +import com.metamx.collections.bitmap.MutableBitmap; +import com.metamx.collections.spatial.ImmutableRTree; +import com.metamx.collections.spatial.RTree; +import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy; +import com.metamx.common.ByteBufferUtils; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import io.druid.collections.CombiningIterable; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ColumnDescriptor; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.BitmapSerdeFactory; +import io.druid.segment.data.ByteBufferWriter; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressedVSizeIndexedV3Writer; +import io.druid.segment.data.CompressedVSizeIntsIndexedWriter; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.GenericIndexedWriter; +import io.druid.segment.data.IOPeon; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedIntsWriter; +import io.druid.segment.data.IndexedRTree; +import io.druid.segment.data.ListIndexed; +import io.druid.segment.data.VSizeIndexedIntsWriter; +import io.druid.segment.data.VSizeIndexedWriter; +import io.druid.segment.serde.DictionaryEncodedColumnPartSerde; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.IntBuffer; +import java.nio.MappedByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +public class StringDimensionMergerV9 implements DimensionMergerV9 +{ + private static final Logger log = new Logger(StringDimensionMergerV9.class); + + protected static final ListIndexed EMPTY_STR_DIM_VAL = new ListIndexed<>(Arrays.asList(""), String.class); + protected static final int[] EMPTY_STR_DIM_ARRAY = new int[]{0}; + protected static final Splitter SPLITTER = Splitter.on(","); + + private IndexedIntsWriter encodedValueWriter; + + protected String dimensionName; + protected GenericIndexedWriter dictionaryWriter; + protected GenericIndexedWriter bitmapWriter; + protected ByteBufferWriter spatialWriter; + protected ArrayList dimConversions; + protected int cardinality = 0; + protected boolean convertMissingValues = false; + protected boolean hasNull = false; + protected MutableBitmap nullRowsBitmap; + protected IOPeon ioPeon; + protected int rowCount = 0; + protected ColumnCapabilities capabilities; + protected final File outDir; + protected List adapters; + protected ProgressIndicator progress; + protected final IndexSpec indexSpec; + + public StringDimensionMergerV9( + String dimensionName, + IndexSpec indexSpec, + File outDir, + IOPeon ioPeon, + ColumnCapabilities capabilities, + ProgressIndicator progress + ) + { + this.dimensionName = dimensionName; + this.indexSpec = indexSpec; + this.capabilities = capabilities; + this.outDir = outDir; + this.ioPeon = ioPeon; + this.progress = progress; + nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); + } + + @Override + public void writeMergedValueMetadata(List adapters) throws IOException + { + boolean dimHasValues = false; + boolean dimAbsentFromSomeIndex = false; + + long dimStartTime = System.currentTimeMillis(); + + this.adapters = adapters; + + dimConversions = Lists.newArrayListWithCapacity(adapters.size()); + for (int i = 0; i < adapters.size(); ++i) { + dimConversions.add(null); + } + + int numMergeIndex = 0; + 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)) { + dimHasValues = true; + hasNull |= dimValues.indexOf(null) >= 0; + dimValueLookups[i] = dimValueLookup = dimValues; + numMergeIndex++; + } else { + dimAbsentFromSomeIndex = true; + } + } + + 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 + * 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. + */ + if (convertMissingValues && !hasNull) { + hasNull = true; + dimValueLookups[adapters.size()] = dimValueLookup = EMPTY_STR_DIM_VAL; + numMergeIndex++; + } + + String dictFilename = String.format("%s.dim_values", dimensionName); + dictionaryWriter = new GenericIndexedWriter<>( + ioPeon, + dictFilename, + GenericIndexed.STRING_STRATEGY + ); + dictionaryWriter.open(); + + cardinality = 0; + if (numMergeIndex > 1) { + IndexMerger.DictionaryMergeIterator iterator = new IndexMerger.DictionaryMergeIterator(dimValueLookups, true); + + while (iterator.hasNext()) { + dictionaryWriter.write(iterator.next()); + } + + for (int i = 0; i < adapters.size(); i++) { + if (dimValueLookups[i] != null && iterator.needConversion(i)) { + dimConversions.set(i, iterator.conversions[i]); + } + } + cardinality = iterator.counter; + } else if (numMergeIndex == 1) { + for (String value : dimValueLookup) { + dictionaryWriter.write(value); + } + cardinality = dimValueLookup.size(); + } + + log.info( + "Completed dim[%s] conversions with cardinality[%,d] in %,d millis.", + dimensionName, + cardinality, + System.currentTimeMillis() - dimStartTime + ); + dictionaryWriter.close(); + + setupEncodedValueWriter(); + } + + protected void setupEncodedValueWriter() throws IOException + { + final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); + + String filenameBase = String.format("%s.forward_dim", dimensionName); + if (capabilities.hasMultipleValues()) { + encodedValueWriter = (compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) + ? CompressedVSizeIndexedV3Writer.create( + ioPeon, + filenameBase, + cardinality, + compressionStrategy + ) + : new VSizeIndexedWriter(ioPeon, filenameBase, cardinality); + } else { + encodedValueWriter = (compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) + ? CompressedVSizeIntsIndexedWriter.create( + ioPeon, + filenameBase, + cardinality, + compressionStrategy + ) + : new VSizeIndexedIntsWriter(ioPeon, filenameBase, cardinality); + } + encodedValueWriter.open(); + } + + + @Override + public int[] convertSegmentRowValuesToMergedRowValues(int[] segmentRow, int segmentIndexNumber) + { + int[] dimVals = (int[]) 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 ? EMPTY_STR_DIM_ARRAY : null; + } + + int[] newDimVals = new int[dimVals.length]; + IntBuffer converter = dimConversions.get(segmentIndexNumber); + + for (int i = 0; i < dimVals.length; i++) { + if (converter != null) { + newDimVals[i] = converter.get(dimVals[i]); + } else { + newDimVals[i] = dimVals[i]; + } + } + + return newDimVals; + } + + @Override + public void processMergedRow(int[] rowValues) throws IOException + { + int[] vals = (int[]) rowValues; + if (vals == null || vals.length == 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. + nullRowsBitmap.add(rowCount); + } + processMergedRowHelper(vals); + rowCount++; + } + + protected void processMergedRowHelper(int[] vals) throws IOException + { + encodedValueWriter.add(vals); + } + + @Override + public void writeIndexes(List segmentRowNumConversions, Closer closer) throws IOException + { + long dimStartTime = System.currentTimeMillis(); + final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); + + String bmpFilename = String.format("%s.inverted", dimensionName); + bitmapWriter = new GenericIndexedWriter<>( + ioPeon, + bmpFilename, + indexSpec.getBitmapSerdeFactory().getObjectStrategy() + ); + bitmapWriter.open(); + + // write dim values to one single file because we need to read it + File dimValueFile = IndexIO.makeDimFile(outDir, dimensionName); + FileOutputStream fos = new FileOutputStream(dimValueFile); + ByteStreams.copy(dictionaryWriter.combineStreams(), fos); + fos.close(); + + final MappedByteBuffer dimValsMapped = Files.map(dimValueFile); + try (Closeable dimValsMappedUnmapper = new Closeable() + { + @Override + public void close() + { + ByteBufferUtils.unmap(dimValsMapped); + } + }) { + Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY); + BitmapFactory bmpFactory = bitmapSerdeFactory.getBitmapFactory(); + + RTree tree = null; + boolean hasSpatial = capabilities.hasSpatialIndexes(); + if (hasSpatial) { + BitmapFactory bitmapFactory = indexSpec.getBitmapSerdeFactory().getBitmapFactory(); + spatialWriter = new ByteBufferWriter<>( + ioPeon, + String.format("%s.spatial", dimensionName), + new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapFactory) + ); + spatialWriter.open(); + tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bmpFactory), bmpFactory); + } + + IndexSeeker[] dictIdSeeker = toIndexSeekers(adapters, dimConversions, dimensionName); + + //Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result. + for (int dictId = 0; dictId < dimVals.size(); dictId++) { + progress.progress(); + List> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size()); + for (int j = 0; j < adapters.size(); ++j) { + int seekedDictId = dictIdSeeker[j].seek(dictId); + if (seekedDictId != IndexSeeker.NOT_EXIST) { + convertedInverteds.add( + new ConvertingIndexedInts( + adapters.get(j).getBitmapIndex(dimensionName, seekedDictId), segmentRowNumConversions.get(j) + ) + ); + } + } + + MutableBitmap bitset = bmpFactory.makeEmptyMutableBitmap(); + for (Integer row : CombiningIterable.createSplatted( + convertedInverteds, + Ordering.natural().nullsFirst() + )) { + if (row != IndexMerger.INVALID_ROW) { + bitset.add(row); + } + } + + ImmutableBitmap bitmapToWrite = bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(bitset); + if ((dictId == 0) && (Iterables.getFirst(dimVals, "") == null)) { + bitmapToWrite = bmpFactory.makeImmutableBitmap(nullRowsBitmap).union(bitmapToWrite); + } + bitmapWriter.write(bitmapToWrite); + + if (hasSpatial) { + String dimVal = dimVals.get(dictId); + if (dimVal != null) { + List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); + float[] coords = new float[stringCoords.size()]; + for (int j = 0; j < coords.length; j++) { + coords[j] = Float.valueOf(stringCoords.get(j)); + } + tree.insert(coords, bitset); + } + } + } + + if (hasSpatial) { + spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); + spatialWriter.close(); + } + + + log.info( + "Completed dim[%s] inverted with cardinality[%,d] in %,d millis.", + dimensionName, + dimVals.size(), + System.currentTimeMillis() - dimStartTime + ); + } + + bitmapWriter.close(); + encodedValueWriter.close(); + + } + + @Override + public boolean canSkip() + { + return cardinality == 0; + } + + @Override + public ColumnDescriptor makeColumnDescriptor() + { + // Now write everything + boolean hasMultiValue = capabilities.hasMultipleValues(); + final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); + final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); + + final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); + builder.setValueType(ValueType.STRING); + builder.setHasMultipleValues(hasMultiValue); + final DictionaryEncodedColumnPartSerde.SerializerBuilder partBuilder = DictionaryEncodedColumnPartSerde + .serializerBuilder() + .withDictionary(dictionaryWriter) + .withValue( + encodedValueWriter, + hasMultiValue, + compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED + ) + .withBitmapSerdeFactory(bitmapSerdeFactory) + .withBitmapIndex(bitmapWriter) + .withSpatialIndex(spatialWriter) + .withByteOrder(IndexIO.BYTE_ORDER); + final ColumnDescriptor serdeficator = builder + .addSerde(partBuilder.build()) + .build(); + + //log.info("Completed dimension column[%s] in %,d millis.", dimensionName, System.currentTimeMillis() - dimStartTime); + + return serdeficator; + } + + protected interface IndexSeeker + { + int NOT_EXIST = -1; + int NOT_INIT = -1; + + int seek(int dictId); + } + + protected class IndexSeekerWithoutConversion implements IndexSeeker + { + private final int limit; + + public IndexSeekerWithoutConversion(int limit) + { + this.limit = limit; + } + + @Override + public int seek(int dictId) + { + return dictId < limit ? dictId : NOT_EXIST; + } + } + + /** + * Get old dictId from new dictId, and only support access in order + */ + protected class IndexSeekerWithConversion implements IndexSeeker + { + private final IntBuffer dimConversions; + private int currIndex; + private int currVal; + private int lastVal; + + IndexSeekerWithConversion(IntBuffer dimConversions) + { + this.dimConversions = dimConversions; + this.currIndex = 0; + this.currVal = NOT_INIT; + this.lastVal = NOT_INIT; + } + + public int seek(int dictId) + { + if (dimConversions == null) { + return NOT_EXIST; + } + if (lastVal != NOT_INIT) { + if (dictId <= lastVal) { + throw new ISE( + "Value dictId[%d] is less than the last value dictId[%d] I have, cannot be.", + dictId, lastVal + ); + } + return NOT_EXIST; + } + if (currVal == NOT_INIT) { + currVal = dimConversions.get(); + } + if (currVal == dictId) { + int ret = currIndex; + ++currIndex; + if (dimConversions.hasRemaining()) { + currVal = dimConversions.get(); + } else { + lastVal = dictId; + } + return ret; + } else if (currVal < dictId) { + throw new ISE( + "Skipped currValue dictId[%d], currIndex[%d]; incoming value dictId[%d]", + currVal, currIndex, dictId + ); + } else { + return NOT_EXIST; + } + } + } + + public static class ConvertingIndexedInts implements Iterable + { + private final IndexedInts baseIndex; + private final IntBuffer conversionBuffer; + + public ConvertingIndexedInts( + IndexedInts baseIndex, + IntBuffer conversionBuffer + ) + { + this.baseIndex = baseIndex; + this.conversionBuffer = conversionBuffer; + } + + public int size() + { + return baseIndex.size(); + } + + public int get(int index) + { + return conversionBuffer.get(baseIndex.get(index)); + } + + @Override + public Iterator iterator() + { + return Iterators.transform( + baseIndex.iterator(), + new Function() + { + @Override + public Integer apply(@Nullable Integer input) + { + return conversionBuffer.get(input); + } + } + ); + } + } + + protected IndexSeeker[] toIndexSeekers( + List adapters, + ArrayList dimConversions, + String dimension + ) + { + IndexSeeker[] seekers = new IndexSeeker[adapters.size()]; + for (int i = 0; i < adapters.size(); i++) { + IntBuffer dimConversion = dimConversions.get(i); + if (dimConversion != null) { + seekers[i] = new IndexSeekerWithConversion((IntBuffer) dimConversion.asReadOnlyBuffer().rewind()); + } else { + Indexed dimValueLookup = (Indexed) adapters.get(i).getDimValueLookup(dimension); + seekers[i] = new IndexSeekerWithoutConversion(dimValueLookup == null ? 0 : dimValueLookup.size()); + } + } + return seekers; + } + + protected boolean isNullColumn(Iterable dimValues) + { + if (dimValues == null) { + return true; + } + for (String val : dimValues) { + if (val != null) { + return false; + } + } + return true; + } +} \ No newline at end of file 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 cb0aaaeac551..22d19b606f91 100644 --- a/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java @@ -25,13 +25,13 @@ /** */ -public interface DictionaryEncodedColumn extends Closeable +public interface DictionaryEncodedColumn extends Closeable { public int length(); public boolean hasMultipleValues(); public int getSingleValueRow(int rowNum); public IndexedInts getMultiValueRow(int rowNum); - public String lookupName(int id); - public int lookupId(String name); + public ActualType lookupName(int id); + public int lookupId(ActualType name); public int getCardinality(); } 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 a7df0c7f7769..c0c5980f94f3 100644 --- a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java @@ -30,7 +30,7 @@ /** */ public class SimpleDictionaryEncodedColumn - implements DictionaryEncodedColumn + implements DictionaryEncodedColumn { private final IndexedInts column; private final IndexedMultivalue multiValueColumn; 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 1b9740b8f597..c8a2fe9b3654 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -21,7 +21,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; -import com.google.common.base.Strings; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -29,12 +28,10 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.metamx.common.IAE; import com.metamx.common.ISE; -import com.metamx.common.parsers.ParseException; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; @@ -47,6 +44,9 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.DimensionHandler; +import io.druid.segment.DimensionHandlerUtil; +import io.druid.segment.DimensionIndexer; import io.druid.segment.DimensionSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; @@ -67,9 +67,9 @@ import javax.annotation.concurrent.GuardedBy; import java.io.Closeable; import java.io.IOException; +import java.lang.reflect.Array; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.Deque; import java.util.Iterator; @@ -101,73 +101,11 @@ public abstract class IncrementalIndex implements Iterable, .put(DimensionSchema.ValueType.STRING, ValueType.STRING) .build(); - private static final Function STRING_TRANSFORMER = new Function() - { - @Override - public String apply(final Object o) - { - return o == null ? null : String.valueOf(o); - } - }; - - private static final Function LONG_TRANSFORMER = new Function() - { - @Override - public Long apply(final Object o) - { - if (o == null) { - return null; - } - if (o instanceof String) { - String s = (String) o; - try { - return s.isEmpty() ? null : Long.valueOf(s); - } - catch (NumberFormatException nfe) { - throw new ParseException(nfe, "Unable to parse value[%s] as long in column: ", o); - } - } - if (o instanceof Number) { - return ((Number) o).longValue(); - } - return null; - } - }; - - private static final Function FLOAT_TRANSFORMER = new Function() - { - @Override - public Float apply(final Object o) - { - if (o == null) { - return null; - } - if (o instanceof String) { - String s = (String) o; - try { - return s.isEmpty() ? null : Float.valueOf(s); - } - catch (NumberFormatException nfe) { - throw new ParseException(nfe, "Unable to parse value[%s] as float in column: ", o); - } - } - if (o instanceof Number) { - return ((Number) o).floatValue(); - } - return null; - } - }; - - private static final Map VALUE_TRANSFORMS = ImmutableMap.builder() - .put(ValueType.LONG, LONG_TRANSFORMER) - .put(ValueType.FLOAT, FLOAT_TRANSFORMER) - .put(ValueType.STRING, STRING_TRANSFORMER) - .build(); - public static ColumnSelectorFactory makeColumnSelectorFactory( final AggregatorFactory agg, final Supplier in, - final boolean deserializeComplexMetrics + final boolean deserializeComplexMetrics, + final Map columnCapabilities ) { return new ColumnSelectorFactory() @@ -369,12 +307,8 @@ public int lookupId(String name) private final Map metricDescs; private final Map dimensionDescs; + private final List dimensionDescsList; private final Map columnCapabilities; - private final List dimValues; - - // looks need a configuration - private final Ordering ordering = Ordering.natural().nullsFirst(); - private final AtomicInteger numEntries = new AtomicInteger(); // This is modified on add() in a critical section. @@ -412,6 +346,7 @@ public IncrementalIndex( this.deserializeComplexMetrics = deserializeComplexMetrics; this.reportParseExceptions = reportParseExceptions; + this.columnCapabilities = Maps.newHashMap(); this.metadata = new Metadata() .setAggregators(getCombiningAggregators(metrics)) .setTimestampSpec(incrementalIndexSchema.getTimestampSpec()) @@ -419,7 +354,6 @@ public IncrementalIndex( .setRollup(this.rollup); this.aggs = initAggs(metrics, rowSupplier, deserializeComplexMetrics); - this.columnCapabilities = Maps.newHashMap(); this.metricDescs = Maps.newLinkedHashMap(); for (AggregatorFactory metric : metrics) { @@ -429,20 +363,20 @@ public IncrementalIndex( } DimensionsSpec dimensionsSpec = incrementalIndexSchema.getDimensionsSpec(); - this.dimensionDescs = Maps.newLinkedHashMap(); - this.dimValues = Collections.synchronizedList(Lists.newArrayList()); + this.dimensionDescsList = new ArrayList<>(); for (DimensionSchema dimSchema : dimensionsSpec.getDimensions()) { - ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl(); ValueType type = TYPE_MAP.get(dimSchema.getValueType()); - capabilities.setType(type); + String dimName = dimSchema.getName(); + ColumnCapabilitiesImpl capabilities = makeCapabilitesFromValueType(type); if (dimSchema.getTypeName().equals(DimensionSchema.SPATIAL_TYPE_NAME)) { capabilities.setHasSpatialIndexes(true); } else { - addNewDimension(dimSchema.getName(), capabilities); + DimensionHandler handler = DimensionHandlerUtil.getHandlerFromCapabilities(dimName, capabilities); + addNewDimension(dimName, capabilities, handler); } - columnCapabilities.put(dimSchema.getName(), capabilities); + columnCapabilities.put(dimName, capabilities); } //__time capabilites @@ -457,28 +391,6 @@ public IncrementalIndex( } } - private DimDim newDimDim(String dimension, ValueType type) - { - DimDim newDimDim; - switch (type) { - case LONG: - newDimDim = makeDimDim(dimension, getDimensionDescs()); - break; - case FLOAT: - newDimDim = makeDimDim(dimension, getDimensionDescs()); - break; - case STRING: - newDimDim = new NullValueConverterDimDim(makeDimDim(dimension, getDimensionDescs())); - break; - default: - throw new IAE("Invalid column type: " + type); - } - return newDimDim; - } - - // use newDimDim() to create a DimDim, makeDimDim() provides the subclass-specific implementation - protected abstract DimDim makeDimDim(String dimension, Object lock); - public boolean isRollup() { return rollup; @@ -521,7 +433,6 @@ protected abstract Integer addToFacts( @Override public void close() { - dimValues.clear(); } public InputRow formatRow(InputRow row) @@ -553,29 +464,14 @@ private ValueType getTypeFromDimVal(Object dimVal) return TYPE_MAP.get(singleVal.getClass()); } - private List getRowDimensionAsComparables(InputRow row, String dimension, ValueType type) + public Map getDimensionDescs() { - final Object dimVal = row.getRaw(dimension); - final Function transformer = VALUE_TRANSFORMS.get(type); - final List dimensionValues; - try { - if (dimVal == null) { - dimensionValues = Collections.emptyList(); - } else if (dimVal instanceof List) { - dimensionValues = Lists.transform((List) dimVal, transformer); - } else { - dimensionValues = Collections.singletonList((Comparable) transformer.apply(dimVal)); - } - } - catch (ParseException pe) { - throw new ParseException(pe.getMessage() + dimension); - } - return dimensionValues; + return dimensionDescs; } - public Map getDimensionDescs() + public Map getColumnCapabilities() { - return dimensionDescs; + return columnCapabilities; } /** @@ -617,45 +513,45 @@ TimeAndDims toTimeAndDims(InputRow row) throws IndexSizeExceededException final List rowDimensions = row.getDimensions(); - int[][] dims; - List overflow = null; - List overflowTypes = null; + Object[] dims; + List overflow = null; synchronized (dimensionDescs) { - dims = new int[dimensionDescs.size()][]; + dims = new Object[dimensionDescs.size()]; for (String dimension : rowDimensions) { - List dimensionValues; - + boolean wasNewDim = false; ColumnCapabilitiesImpl capabilities; - final ValueType valType; + ValueType valType = null; DimensionDesc desc = dimensionDescs.get(dimension); if (desc != null) { capabilities = desc.getCapabilities(); } else { + wasNewDim = true; capabilities = columnCapabilities.get(dimension); if (capabilities == null) { capabilities = new ColumnCapabilitiesImpl(); // For schemaless type discovery, assume everything is a String for now, can change later. capabilities.setType(ValueType.STRING); + capabilities.setDictionaryEncoded(true); + capabilities.setHasBitmapIndexes(true); columnCapabilities.put(dimension, capabilities); } + DimensionHandler handler = DimensionHandlerUtil.getHandlerFromCapabilities(dimension, capabilities); + desc = addNewDimension(dimension, capabilities, handler); } - valType = capabilities.getType(); - dimensionValues = getRowDimensionAsComparables(row, dimension, valType); + DimensionHandler handler = desc.getHandler(); + DimensionIndexer indexer = desc.getIndexer(); + Object dimsKey = indexer.processRowValsToUnsortedEncodedArray(row.getRaw(dimension)); // Set column capabilities as data is coming in - if (!capabilities.hasMultipleValues() && dimensionValues.size() > 1) { + if (!capabilities.hasMultipleValues() && dimsKey != null && handler.getLengthFromEncodedArray(dimsKey) > 1) { capabilities.setHasMultipleValues(true); } - if (desc == null) { - desc = addNewDimension(dimension, capabilities); - + if (wasNewDim) { if (overflow == null) { overflow = Lists.newArrayList(); - overflowTypes = Lists.newArrayList(); } - overflow.add(getDimVals(desc.getValues(), dimensionValues)); - overflowTypes.add(valType); + overflow.add(dimsKey); } else if (desc.getIndex() > dims.length || dims[desc.getIndex()] != null) { /* * index > dims.length requires that we saw this dimension and added it to the dimensionOrder map, @@ -668,14 +564,14 @@ TimeAndDims toTimeAndDims(InputRow row) throws IndexSizeExceededException */ throw new ISE("Dimension[%s] occurred more than once in InputRow", dimension); } else { - dims[desc.getIndex()] = getDimVals(desc.getValues(), dimensionValues); + dims[desc.getIndex()] = dimsKey; } } } if (overflow != null) { // Merge overflow and non-overflow - int[][] newDims = new int[dims.length + overflow.size()][]; + Object[] newDims = new Object[dims.length + overflow.size()]; System.arraycopy(dims, 0, newDims, 0, dims.length); for (int i = 0; i < overflow.size(); ++i) { newDims[dims.length + i] = overflow.get(i); @@ -684,7 +580,7 @@ TimeAndDims toTimeAndDims(InputRow row) throws IndexSizeExceededException } long truncated = gran.truncate(row.getTimestampFromEpoch()); - return new TimeAndDims(Math.max(truncated, minTimestamp), dims); + return new TimeAndDims(Math.max(truncated, minTimestamp), dims, dimensionDescsList); } private synchronized void updateMaxIngestedTime(DateTime eventTime) @@ -714,33 +610,6 @@ private long getMaxTimeMillis() return getFacts().getMaxTimeMillis(); } - private int[] getDimVals(final DimDim dimLookup, final List dimValues) - { - if (dimValues.size() == 0) { - // NULL VALUE - dimLookup.add(null); - return null; - } - - if (dimValues.size() == 1) { - Comparable dimVal = dimValues.get(0); - // For Strings, return an array of dictionary-encoded IDs - // For numerics, return the numeric values directly - return new int[]{dimLookup.add(dimVal)}; - } - - Comparable[] dimArray = dimValues.toArray(new Comparable[dimValues.size()]); - Arrays.sort(dimArray, ordering); - - final int[] retVal = new int[dimArray.length]; - - for (int i = 0; i < dimArray.length; i++) { - retVal[i] = dimLookup.add(dimArray[i]); - } - - return retVal; - } - public AggregatorType[] getAggs() { return aggs; @@ -809,10 +678,10 @@ public DateTime getMaxTime() return isEmpty() ? null : new DateTime(getMaxTimeMillis()); } - public DimDim getDimensionValues(String dimension) + public Integer getDimensionIndex(String dimension) { DimensionDesc dimSpec = getDimension(dimension); - return dimSpec == null ? null : dimSpec.getValues(); + return dimSpec == null ? null : dimSpec.getIndex(); } public List getDimensionOrder() @@ -822,12 +691,21 @@ public List getDimensionOrder() } } + private ColumnCapabilitiesImpl makeCapabilitesFromValueType(ValueType type) + { + ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl(); + capabilities.setDictionaryEncoded(type == ValueType.STRING); + capabilities.setHasBitmapIndexes(type == ValueType.STRING); + capabilities.setType(type); + return capabilities; + } + /* * Currently called to initialize IncrementalIndex dimension order during index creation * Index dimension ordering could be changed to initialize from DimensionsSpec after resolution of * https://github.com/druid-io/druid/issues/2011 */ - public void loadDimensionIterable(Iterable oldDimensionOrder) + public void loadDimensionIterable(Iterable oldDimensionOrder, Map oldColumnCapabilities) { synchronized (dimensionDescs) { if (!dimensionDescs.isEmpty()) { @@ -835,25 +713,21 @@ public void loadDimensionIterable(Iterable oldDimensionOrder) } for (String dim : oldDimensionOrder) { if (dimensionDescs.get(dim) == null) { - ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl(); - capabilities.setType(ValueType.STRING); + ColumnCapabilitiesImpl capabilities = oldColumnCapabilities.get(dim); columnCapabilities.put(dim, capabilities); - addNewDimension(dim, capabilities); + DimensionHandler handler = DimensionHandlerUtil.getHandlerFromCapabilities(dim, capabilities); + addNewDimension(dim, capabilities, handler); } } } } @GuardedBy("dimensionDescs") - private DimensionDesc addNewDimension(String dim, ColumnCapabilitiesImpl capabilities) + private DimensionDesc addNewDimension(String dim, ColumnCapabilitiesImpl capabilities, DimensionHandler handler) { - DimensionDesc desc = new DimensionDesc(dimensionDescs.size(), dim, newDimDim(dim, capabilities.getType()), capabilities); - if (dimValues.size() != desc.getIndex()) { - throw new ISE("dimensionDescs and dimValues for [%s] is out of sync!!", dim); - } - + DimensionDesc desc = new DimensionDesc(dimensionDescs.size(), dim, capabilities, handler); dimensionDescs.put(dim, desc); - dimValues.add(desc.getValues()); + dimensionDescsList.add(desc); return desc; } @@ -892,6 +766,15 @@ 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() { @@ -917,38 +800,24 @@ public Row apply(final Map.Entry input) final TimeAndDims timeAndDims = input.getKey(); final int rowOffset = input.getValue(); - int[][] theDims = timeAndDims.getDims(); //TODO: remove dictionary encoding for numerics later + Object[] theDims = timeAndDims.getDims(); Map theVals = Maps.newLinkedHashMap(); for (int i = 0; i < theDims.length; ++i) { - int[] dim = theDims[i]; + Object dim = theDims[i]; DimensionDesc dimensionDesc = dimensions.get(i); if (dimensionDesc == null) { continue; } - ValueType type = dimensionDesc.getCapabilities().getType(); String dimensionName = dimensionDesc.getName(); - if (dim == null || dim.length == 0) { + DimensionHandler handler = dimensionDesc.getHandler(); + if (dim == null || handler.getLengthFromEncodedArray(dim) == 0) { theVals.put(dimensionName, null); continue; } - if (dim.length == 1) { - Comparable val = dimensionDesc.getValues().getValue(dim[0]); - if (type == ValueType.STRING) { - val = Strings.nullToEmpty((String) val); - } - theVals.put(dimensionName, val); - } else { - List dimVals = new ArrayList(dim.length); - for (int j = 0; j < dim.length; j++) { - Comparable val = dimensionDesc.getValues().getValue(dim[j]); - if (type == ValueType.STRING) { - val = Strings.nullToEmpty((String) val); - } - dimVals.add(val); - } - theVals.put(dimensionName, dimVals); - } + final DimensionIndexer indexer = dimensionDesc.getIndexer(); + Object rowVals = indexer.convertUnsortedEncodedArrayToActualArrayOrList(dim, true); + theVals.put(dimensionName, rowVals); } AggregatorType[] aggs = getAggsForRow(rowOffset); @@ -979,15 +848,17 @@ public static final class DimensionDesc { private final int index; private final String name; - private final DimDim values; private final ColumnCapabilitiesImpl capabilities; + private final DimensionHandler handler; + private final DimensionIndexer indexer; - public DimensionDesc(int index, String name, DimDim values, ColumnCapabilitiesImpl capabilities) + public DimensionDesc(int index, String name, ColumnCapabilitiesImpl capabilities, DimensionHandler handler) { this.index = index; this.name = name; - this.values = values; this.capabilities = capabilities; + this.handler = handler; + this.indexer = handler.makeIndexer(); } public int getIndex() @@ -1000,14 +871,19 @@ public String getName() return name; } - public DimDim getValues() + public ColumnCapabilitiesImpl getCapabilities() { - return values; + return capabilities; } - public ColumnCapabilitiesImpl getCapabilities() + public DimensionHandler getHandler() { - return capabilities; + return handler; + } + + public DimensionIndexer getIndexer() + { + return indexer; } } @@ -1058,151 +934,29 @@ public ColumnCapabilitiesImpl getCapabilities() } } - static interface DimDim> - { - public int getId(T value); - - public T getValue(int id); - - public boolean contains(T value); - - public int size(); - - public T getMinValue(); - - public T getMaxValue(); - - public int add(T value); - - public SortedDimLookup sort(); - } - - static interface SortedDimLookup> - { - public int size(); - - public int getSortedIdFromUnsortedId(int id); - - public int getUnsortedIdFromSortedId(int index); - - public T getValueFromSortedId(int index); - } - - /** - * implementation which converts null strings to empty strings and vice versa. - */ - static class NullValueConverterDimDim implements DimDim - { - private final DimDim delegate; - - NullValueConverterDimDim(DimDim delegate) - { - this.delegate = delegate; - } - - @Override - public int getId(String value) - { - return delegate.getId(Strings.nullToEmpty(value)); - } - - @Override - public String getValue(int id) - { - return Strings.emptyToNull(delegate.getValue(id)); - } - - @Override - public boolean contains(String value) - { - return delegate.contains(Strings.nullToEmpty(value)); - } - - @Override - public int size() - { - return delegate.size(); - } - - @Override - public String getMinValue() - { - return Strings.nullToEmpty(delegate.getMinValue()); - } - - @Override - public String getMaxValue() - { - return Strings.nullToEmpty(delegate.getMaxValue()); - } - - @Override - public int add(String value) - { - return delegate.add(Strings.nullToEmpty(value)); - } - - @Override - public SortedDimLookup sort() - { - return new NullValueConverterDimLookup(delegate.sort()); - } - } - - private static class NullValueConverterDimLookup implements SortedDimLookup - { - private final SortedDimLookup delegate; - - public NullValueConverterDimLookup(SortedDimLookup delegate) - { - this.delegate = delegate; - } - - @Override - public int size() - { - return delegate.size(); - } - - @Override - public int getUnsortedIdFromSortedId(int index) - { - return delegate.getUnsortedIdFromSortedId(index); - } - - @Override - public int getSortedIdFromUnsortedId(int id) - { - return delegate.getSortedIdFromUnsortedId(id); - } - - @Override - public String getValueFromSortedId(int index) - { - return Strings.emptyToNull(delegate.getValueFromSortedId(index)); - } - } - - static final class TimeAndDims + public static final class TimeAndDims { private final long timestamp; - private final int[][] dims; + private final Object[] dims; + private final List dimensionDescsList; TimeAndDims( long timestamp, - int[][] dims + Object[] dims, + List dimensionDescsList ) { this.timestamp = timestamp; this.dims = dims; + this.dimensionDescsList = dimensionDescsList; } - long getTimestamp() + public long getTimestamp() { return timestamp; } - int[][] getDims() + public Object[] getDims() { return dims; } @@ -1213,12 +967,12 @@ public String toString() return "TimeAndDims{" + "timestamp=" + new DateTime(timestamp) + ", dims=" + Lists.transform( - Arrays.asList(dims), new Function() + Arrays.asList(dims), new Function() { @Override - public Object apply(@Nullable int[] input) + public Object apply(@Nullable Object input) { - if (input == null || input.length == 0) { + if (input == null || Array.getLength(input) == 0) { return Arrays.asList("null"); } return Arrays.asList(input); @@ -1246,7 +1000,8 @@ public boolean equals(Object o) return false; } for (int i = 0; i < dims.length; i++) { - if (!Arrays.equals(dims[i], that.dims[i])) { + final DimensionIndexer indexer = dimensionDescsList.get(i).getIndexer(); + if (!indexer.checkUnsortedEncodedArraysEqual(dims[i], that.dims[i])) { return false; } } @@ -1258,7 +1013,8 @@ public int hashCode() { int hash = (int) timestamp; for (int i = 0; i < dims.length; i++) { - hash = 31 * hash + Arrays.hashCode(dims[i]); + final DimensionIndexer indexer = dimensionDescsList.get(i).getIndexer(); + hash = 31 * hash + indexer.getUnsortedEncodedArrayHashCode(dims[i]); } return hash; } @@ -1266,17 +1022,17 @@ public int hashCode() protected final Comparator dimsComparator() { - return new TimeAndDimsComp(dimValues); + return new TimeAndDimsComp(dimensionDescsList); } @VisibleForTesting static final class TimeAndDimsComp implements Comparator { - private final List dimValues; + private List dimensionDescs; - public TimeAndDimsComp(List dimValues) + public TimeAndDimsComp(List dimDescs) { - this.dimValues = dimValues; + this.dimensionDescs = dimDescs; } @Override @@ -1287,8 +1043,8 @@ public int compare(TimeAndDims lhs, TimeAndDims rhs) int index = 0; while (retVal == 0 && index < numComparisons) { - final int[] lhsIdxs = lhs.dims[index]; - final int[] rhsIdxs = rhs.dims[index]; + final Object lhsIdxs = lhs.dims[index]; + final Object rhsIdxs = rhs.dims[index]; if (lhsIdxs == null) { if (rhsIdxs == null) { @@ -1302,22 +1058,8 @@ public int compare(TimeAndDims lhs, TimeAndDims rhs) return 1; } - retVal = Ints.compare(lhsIdxs.length, rhsIdxs.length); - - int valsIndex = 0; - while (retVal == 0 && valsIndex < lhsIdxs.length) { - if (lhsIdxs[valsIndex] != rhsIdxs[valsIndex]) { - final DimDim dimLookup = dimValues.get(index); - final Comparable lhsVal = dimLookup.getValue(lhsIdxs[valsIndex]); - final Comparable rhsVal = dimLookup.getValue(rhsIdxs[valsIndex]); - if (lhsVal != null && rhsVal != null) { - retVal = lhsVal.compareTo(rhsVal); - } else if (lhsVal == null ^ rhsVal == null) { - retVal = lhsVal == null ? -1 : 1; - } - } - ++valsIndex; - } + final DimensionIndexer indexer = dimensionDescs.get(index).getIndexer(); + retVal = indexer.compareUnsortedEncodedArrays(lhsIdxs, rhsIdxs); ++index; } @@ -1416,8 +1158,9 @@ static class RollupFactsHolder implements FactsHolder { private final boolean sortFacts; private final ConcurrentMap facts; + private final List dimensionDescsList; - public RollupFactsHolder(boolean sortFacts, Comparator timeAndDimsComparator) + public RollupFactsHolder(boolean sortFacts, Comparator timeAndDimsComparator, List dimensionDescsList) { this.sortFacts = sortFacts; if (sortFacts) { @@ -1425,6 +1168,7 @@ public RollupFactsHolder(boolean sortFacts, Comparator timeAndDimsC } else { this.facts = new ConcurrentHashMap<>(); } + this.dimensionDescsList = dimensionDescsList; } @Override @@ -1473,8 +1217,8 @@ public Iterable> timeRangeIterable(boolean desce if (!sortFacts) { throw new UnsupportedOperationException("can't get timeRange from unsorted facts data."); } - TimeAndDims start = new TimeAndDims(timeStart, new int[][]{}); - TimeAndDims end = new TimeAndDims(timeEnd, new int[][]{}); + 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; 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 7d5572ca73ae..cc180b519d5a 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java @@ -22,10 +22,11 @@ import com.google.common.base.Function; import com.google.common.collect.Iterators; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import com.metamx.collections.bitmap.BitmapFactory; import com.metamx.collections.bitmap.MutableBitmap; import com.metamx.common.logger.Logger; +import io.druid.segment.DimensionHandler; +import io.druid.segment.DimensionIndexer; import io.druid.segment.IndexableAdapter; import io.druid.segment.Metadata; import io.druid.segment.Rowboat; @@ -33,7 +34,6 @@ import io.druid.segment.data.EmptyIndexedInts; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; -import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.ListIndexed; import org.joda.time.Interval; import org.roaringbitmap.IntIterator; @@ -42,7 +42,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; /** */ @@ -51,38 +50,23 @@ public class IncrementalIndexAdapter implements IndexableAdapter private static final Logger log = new Logger(IncrementalIndexAdapter.class); private final Interval dataInterval; private final IncrementalIndex index; - private final Set hasNullValueDimensions; + private final Map accessors; - private final Map indexers; - - private class DimensionIndexer + private class DimensionAccessor { private final IncrementalIndex.DimensionDesc dimensionDesc; private final MutableBitmap[] invertedIndexes; + private final DimensionIndexer indexer; - private IncrementalIndex.SortedDimLookup dimLookup; - - public DimensionIndexer(IncrementalIndex.DimensionDesc dimensionDesc) + public DimensionAccessor(IncrementalIndex.DimensionDesc dimensionDesc) { this.dimensionDesc = dimensionDesc; - this.invertedIndexes = new MutableBitmap[dimensionDesc.getValues().size() + 1]; - } - - private IncrementalIndex.DimDim getDimValues() - { - return dimensionDesc.getValues(); - } - - private IncrementalIndex.SortedDimLookup getDimLookup() - { - if (dimLookup == null) { - final IncrementalIndex.DimDim dimDim = dimensionDesc.getValues(); - if (hasNullValueDimensions.contains(dimensionDesc.getName()) && !dimDim.contains(null)) { - dimDim.add(null); - } - dimLookup = dimDim.sort(); + this.indexer = dimensionDesc.getIndexer(); + if(dimensionDesc.getCapabilities().hasBitmapIndexes()) { + this.invertedIndexes = new MutableBitmap[indexer.getCardinality() + 1]; + } else { + this.invertedIndexes = null; } - return dimLookup; } } @@ -102,42 +86,34 @@ public IncrementalIndexAdapter( * in row 2. So we should iterate all rows to determine whether one dimension contains * a null value. */ - this.hasNullValueDimensions = Sets.newHashSet(); - final List dimensions = index.getDimensions(); - indexers = Maps.newHashMapWithExpectedSize(dimensions.size()); + accessors = Maps.newHashMapWithExpectedSize(dimensions.size()); for (IncrementalIndex.DimensionDesc dimension : dimensions) { - indexers.put(dimension.getName(), new DimensionIndexer(dimension)); + accessors.put(dimension.getName(), new DimensionAccessor(dimension)); } int rowNum = 0; for (IncrementalIndex.TimeAndDims timeAndDims : index.getFacts().keySet()) { - final int[][] dims = timeAndDims.getDims(); + final Object[] dims = timeAndDims.getDims(); for (IncrementalIndex.DimensionDesc dimension : dimensions) { final int dimIndex = dimension.getIndex(); - DimensionIndexer indexer = indexers.get(dimension.getName()); + DimensionAccessor accessor = accessors.get(dimension.getName()); + + // Add 'null' to the dimension's dictionary. if (dimIndex >= dims.length || dims[dimIndex] == null) { - hasNullValueDimensions.add(dimension.getName()); + accessor.indexer.processRowValsToUnsortedEncodedArray(null); continue; } - final IncrementalIndex.DimDim values = dimension.getValues(); - if (hasNullValue(values, dims[dimIndex])) { - hasNullValueDimensions.add(dimension.getName()); - } + final ColumnCapabilities capabilities = dimension.getCapabilities(); - final MutableBitmap[] bitmapIndexes = indexer.invertedIndexes; - - for (Comparable dimIdxComparable : dims[dimIndex]) { - Integer dimIdx = (Integer) dimIdxComparable; - if (bitmapIndexes[dimIdx] == null) { - bitmapIndexes[dimIdx] = bitmapFactory.makeEmptyMutableBitmap(); - } - bitmapIndexes[dimIdx].add(rowNum); + if(capabilities.hasBitmapIndexes()) { + final MutableBitmap[] bitmapIndexes = accessor.invertedIndexes; + final DimensionIndexer indexer = accessor.indexer; + indexer.fillBitmapsFromUnsortedEncodedArray(dims[dimIndex], rowNum, bitmapIndexes, bitmapFactory); } } - ++rowNum; } } @@ -167,50 +143,16 @@ public Indexed getMetricNames() } @Override - public Indexed getDimValueLookup(String dimension) + public Indexed getDimValueLookup(String dimension) { - final DimensionIndexer indexer = indexers.get(dimension); - if (indexer == null) { + final DimensionAccessor accessor = accessors.get(dimension); + if (accessor == null) { return null; } - final IncrementalIndex.DimDim dimDim = indexer.getDimValues(); - final IncrementalIndex.SortedDimLookup dimLookup = indexer.getDimLookup(); - return new Indexed() - { - @Override - public Class getClazz() - { - return String.class; - } - - @Override - public int size() - { - return dimLookup.size(); - } + final DimensionIndexer indexer = accessor.dimensionDesc.getIndexer(); - @Override - public String get(int index) - { - Comparable val = dimLookup.getValueFromSortedId(index); - String strVal = val != null ? val.toString() : null; - return strVal; - } - - @Override - public int indexOf(String value) - { - int id = dimDim.getId(value); - return id < 0 ? -1 : dimLookup.getSortedIdFromUnsortedId(id); - } - - @Override - public Iterator iterator() - { - return IndexedIterable.create(this).iterator(); - } - }; + return indexer.getSortedIndexedValues(); } @Override @@ -222,9 +164,11 @@ public Iterable getRows() public Iterator iterator() { final List dimensions = index.getDimensions(); - final IncrementalIndex.SortedDimLookup[] sortedDimLookups = new IncrementalIndex.SortedDimLookup[dimensions.size()]; + final DimensionHandler[] handlers = new DimensionHandler[dimensions.size()]; + final DimensionIndexer[] indexers = new DimensionIndexer[dimensions.size()]; for (IncrementalIndex.DimensionDesc dimension : dimensions) { - sortedDimLookups[dimension.getIndex()] = indexers.get(dimension.getName()).getDimLookup(); + handlers[dimension.getIndex()] = dimension.getHandler(); + indexers[dimension.getIndex()] = dimension.getIndexer(); } /* @@ -242,10 +186,10 @@ public Iterator iterator() public Rowboat apply(Map.Entry input) { final IncrementalIndex.TimeAndDims timeAndDims = input.getKey(); - final int[][] dimValues = timeAndDims.getDims(); + final Object[] dimValues = timeAndDims.getDims(); final int rowOffset = input.getValue(); - int[][] dims = new int[dimValues.length][]; + Object[] dims = new Object[dimValues.length]; for (IncrementalIndex.DimensionDesc dimension : dimensions) { final int dimIndex = dimension.getIndex(); @@ -253,17 +197,9 @@ public Rowboat apply(Map.Entry input) continue; } - dims[dimIndex] = new int[dimValues[dimIndex].length]; - - if (dimIndex >= dims.length || dims[dimIndex] == null) { - continue; - } - - for (int i = 0; i < dimValues[dimIndex].length; ++i) { - dims[dimIndex][i] = sortedDimLookups[dimIndex].getSortedIdFromUnsortedId(dimValues[dimIndex][i]); - //TODO: in later PR, Rowboat will use Comparable[][] instead of int[][] - // Can remove dictionary encoding for numeric dims then. - } + final DimensionIndexer indexer = indexers[dimIndex]; + Object sortedDimVals = indexer.convertUnsortedEncodedArrayToSortedEncodedArray(dimValues[dimIndex]); + dims[dimIndex] = sortedDimVals; } Object[] metrics = new Object[index.getMetricAggs().length]; @@ -275,7 +211,8 @@ public Rowboat apply(Map.Entry input) timeAndDims.getTimestamp(), dims, metrics, - count++ + count++, + handlers ); } } @@ -287,14 +224,19 @@ public Rowboat apply(Map.Entry input) @Override public IndexedInts getBitmapIndex(String dimension, int index) { - DimensionIndexer accessor = indexers.get(dimension); + DimensionAccessor accessor = accessors.get(dimension); if (accessor == null) { return EmptyIndexedInts.EMPTY_INDEXED_INTS; } + ColumnCapabilities capabilities = accessor.dimensionDesc.getCapabilities(); + DimensionIndexer indexer = accessor.dimensionDesc.getIndexer(); + + if (!capabilities.hasBitmapIndexes()) { + return EmptyIndexedInts.EMPTY_INDEXED_INTS; + } - IncrementalIndex.SortedDimLookup dimLookup = accessor.getDimLookup(); - final int id = dimLookup.getUnsortedIdFromSortedId(index); - if (id < 0 || id >= dimLookup.size()) { + final int id = (Integer) indexer.getUnsortedEncodedValueFromSorted(index); + if (id < 0 || id >= indexer.getCardinality()) { return EmptyIndexedInts.EMPTY_INDEXED_INTS; } @@ -319,27 +261,6 @@ public ColumnCapabilities getCapabilities(String column) return index.getCapabilities(column); } - private boolean hasNullValue(IncrementalIndex.DimDim dimDim, int[] dimIndices) - { - if (dimIndices == null || dimIndices.length == 0) { - return true; - } - for (int dimIndex : dimIndices) { - Comparable val = dimDim.getValue(dimIndex); - - if (val == null) { - return true; - } - - if (val instanceof String) { - if (((String) val).length() == 0) { - return true; - } - } - } - return false; - } - static class BitmapIndexedInts implements IndexedInts { @@ -407,4 +328,10 @@ public Metadata getMetadata() { return index.getMetadata(); } + + @Override + public Map getDimensionHandlers() + { + return index.getDimensionHandlers(); + } } 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 941fa15f75d3..0af3152c4ca1 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -20,16 +20,15 @@ package io.druid.segment.incremental; import com.google.common.base.Function; -import com.google.common.base.Predicate; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; -import com.google.common.primitives.Ints; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.granularity.QueryGranularity; import io.druid.query.QueryInterruptedException; +import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.DruidLongPredicate; @@ -39,6 +38,8 @@ import io.druid.query.filter.ValueMatcherFactory; import io.druid.segment.Capabilities; import io.druid.segment.Cursor; +import io.druid.segment.DimensionHandler; +import io.druid.segment.DimensionIndexer; import io.druid.segment.DimensionSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; @@ -51,7 +52,6 @@ import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; import io.druid.segment.data.Indexed; -import io.druid.segment.data.IndexedInts; import io.druid.segment.data.ListIndexed; import io.druid.segment.filter.BooleanValueMatcher; import io.druid.segment.filter.Filters; @@ -59,11 +59,7 @@ import org.joda.time.Interval; import javax.annotation.Nullable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.Iterator; -import java.util.List; import java.util.Map; /** @@ -111,11 +107,14 @@ public int getDimensionCardinality(String dimension) if (dimension.equals(Column.TIME_COLUMN_NAME)) { return Integer.MAX_VALUE; } - IncrementalIndex.DimDim dimDim = index.getDimensionValues(dimension); - if (dimDim == null) { + + IncrementalIndex.DimensionDesc desc = index.getDimension(dimension); + if (desc == null) { return 0; } - return dimDim.size(); + + DimensionIndexer indexer = index.getDimension(dimension).getIndexer(); + return indexer.getCardinality(); } @Override @@ -139,15 +138,25 @@ public DateTime getMaxTime() @Override public Comparable getMinValue(String column) { - IncrementalIndex.DimDim dimDim = index.getDimensionValues(column); - return dimDim == null ? null : dimDim.getMinValue(); + IncrementalIndex.DimensionDesc desc = index.getDimension(column); + if (desc == null) { + return null; + } + + DimensionIndexer indexer = desc.getIndexer(); + return indexer.getMinValue(); } @Override public Comparable getMaxValue(String column) { - IncrementalIndex.DimDim dimDim = index.getDimensionValues(column); - return dimDim == null ? null : dimDim.getMaxValue(); + IncrementalIndex.DimensionDesc desc = index.getDimension(column); + if (desc == null) { + return null; + } + + DimensionIndexer indexer = desc.getIndexer(); + return indexer.getMaxValue(); } @Override @@ -162,6 +171,12 @@ public ColumnCapabilities getColumnCapabilities(String column) return index.getCapabilities(column); } + @Override + public Map getDimensionHandlers() + { + return index.getDimensionHandlers(); + } + @Override public String getColumnTypeName(String column) { @@ -176,7 +191,12 @@ public DateTime getMaxIngestedEventTime() } @Override - public Sequence makeCursors(final Filter filter, final Interval interval, final QueryGranularity gran, final boolean descending) + public Sequence makeCursors( + final Filter filter, + final Interval interval, + final QueryGranularity gran, + final boolean descending + ) { if (index.isEmpty()) { return Sequences.empty(); @@ -320,124 +340,42 @@ public void reset() public DimensionSelector makeDimensionSelector( DimensionSpec dimensionSpec ) - { - return dimensionSpec.decorate(makeDimensionSelectorUndecorated(dimensionSpec)); - } - - private DimensionSelector makeDimensionSelectorUndecorated( - DimensionSpec dimensionSpec - ) { final String dimension = dimensionSpec.getDimension(); final ExtractionFn extractionFn = dimensionSpec.getExtractionFn(); if (dimension.equals(Column.TIME_COLUMN_NAME)) { - return new SingleScanTimeDimSelector(makeLongColumnSelector(dimension), extractionFn, descending); + DimensionSelector selector = new SingleScanTimeDimSelector( + makeLongColumnSelector(dimension), + extractionFn, + descending + ); + return dimensionSpec.decorate(selector); } - final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimension); + final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimensionSpec.getDimension()); if (dimensionDesc == null) { - return NULL_DIMENSION_SELECTOR; + return dimensionSpec.decorate(NULL_DIMENSION_SELECTOR); } - final int dimIndex = dimensionDesc.getIndex(); - final IncrementalIndex.DimDim dimValLookup = dimensionDesc.getValues(); - - final int maxId = dimValLookup.size(); - - return new DimensionSelector() - { - @Override - public IndexedInts getRow() - { - final int[][] dims = currEntry.getKey().getDims(); - - int[] indices = dimIndex < dims.length ? dims[dimIndex] : null; - - List valsTmp = null; - if ((indices == null || indices.length == 0) && dimValLookup.contains(null)) { - int id = dimValLookup.getId(null); - if (id < maxId) { - valsTmp = new ArrayList<>(1); - valsTmp.add(id); - } - } else if (indices != null && indices.length > 0) { - valsTmp = new ArrayList<>(indices.length); - for (int i = 0; i < indices.length; i++) { - int id = indices[i]; - if (id < maxId) { - valsTmp.add(id); - } - } - } - - final List vals = valsTmp == null ? Collections.EMPTY_LIST : valsTmp; - return new IndexedInts() - { - @Override - public int size() - { - return vals.size(); - } - - @Override - public int get(int index) - { - return vals.get(index); - } - - @Override - public Iterator iterator() - { - return vals.iterator(); - } - - @Override - public void fill(int index, int[] toFill) - { - throw new UnsupportedOperationException("fill not supported"); - } - - @Override - public void close() throws IOException - { - - } - }; - } - - @Override - public int getValueCardinality() - { - return maxId; - } - - @Override - public String lookupName(int id) - { - // TODO: needs update to DimensionSelector interface to allow multi-types, just use Strings for now - final Comparable value = dimValLookup.getValue(id); - final String strValue = value == null ? null : value.toString(); - return extractionFn == null ? strValue : extractionFn.apply(strValue); - - } - - @Override - public int lookupId(String name) - { - if (extractionFn != null) { - throw new UnsupportedOperationException( - "cannot perform lookup when applying an extraction function" - ); - } - return dimValLookup.getId(name); - } - }; + final DimensionIndexer indexer = dimensionDesc.getIndexer(); + return dimensionSpec.decorate((DimensionSelector) indexer.makeColumnValueSelector(dimensionSpec, currEntry, dimensionDesc)); } @Override public FloatColumnSelector makeFloatColumnSelector(String columnName) { + final Integer dimIndex = index.getDimensionIndex(columnName); + if (dimIndex != null) { + final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(columnName); + final DimensionIndexer indexer = dimensionDesc.getIndexer(); + return (FloatColumnSelector) indexer.makeColumnValueSelector( + new DefaultDimensionSpec(columnName, null), + currEntry, + dimensionDesc + ); + } + final Integer metricIndexInt = index.getMetricIndex(columnName); if (metricIndexInt == null) { return new FloatColumnSelector() @@ -474,6 +412,18 @@ public long get() } }; } + + final Integer dimIndex = index.getDimensionIndex(columnName); + if (dimIndex != null) { + final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(columnName); + final DimensionIndexer indexer = dimensionDesc.getIndexer(); + return (LongColumnSelector) indexer.makeColumnValueSelector( + new DefaultDimensionSpec(columnName, null), + currEntry, + dimensionDesc + ); + } + final Integer metricIndexInt = index.getMetricIndex(columnName); if (metricIndexInt == null) { return new LongColumnSelector() @@ -549,7 +499,8 @@ public Object get() if (dimensionDesc != null) { final int dimensionIndex = dimensionDesc.getIndex(); - final IncrementalIndex.DimDim dimDim = dimensionDesc.getValues(); + final DimensionIndexer indexer = dimensionDesc.getIndexer(); + final ColumnCapabilities capabilities = dimensionDesc.getCapabilities(); return new ObjectColumnSelector() { @@ -567,22 +518,12 @@ public Object get() return null; } - int[][] dims = key.getDims(); + Object[] dims = key.getDims(); if (dimensionIndex >= dims.length) { return null; } - final int[] dimIdx = dims[dimensionIndex]; - if (dimIdx == null || dimIdx.length == 0) { - return null; - } - if (dimIdx.length == 1) { - return dimDim.getValue(dimIdx[0]); - } - Comparable[] dimVals = new String[dimIdx.length]; - for (int i = 0; i < dimIdx.length; i++) { - dimVals[i] = dimDim.getValue(dimIdx[i]); - } + Object dimVals = indexer.convertUnsortedEncodedArrayToActualArrayOrList(dims[dimensionIndex], false); return dimVals; } }; @@ -617,7 +558,7 @@ private ValueMatcher makeFilterMatcher(final Filter filter, final Cursor cursor, : filter.makeMatcher(new CursorAndEntryHolderValueMatcherFactory(cursor, holder)); } - private static class EntryHolder + public static class EntryHolder { Map.Entry currEntry = null; @@ -658,96 +599,56 @@ public CursorAndEntryHolderValueMatcherFactory( } @Override - public ValueMatcher makeValueMatcher(String dimension, final Comparable value) + public ValueMatcher makeValueMatcher(String dimension, final Comparable originalValue) { - if (getTypeForDimension(dimension) == ValueType.LONG) { - return Filters.getLongValueMatcher(cursor.makeLongColumnSelector(dimension), value); - } - IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimension); if (dimensionDesc == null) { - return new BooleanValueMatcher(isComparableNullOrEmpty(value)); - } - final int dimIndex = dimensionDesc.getIndex(); - final IncrementalIndex.DimDim dimDim = dimensionDesc.getValues(); - - final Integer id = dimDim.getId(value); - if (id == null) { - if (isComparableNullOrEmpty(value)) { - return new ValueMatcher() - { - @Override - public boolean matches() - { - int[][] dims = holder.getKey().getDims(); - if (dimIndex >= dims.length || dims[dimIndex] == null) { - return true; - } - return false; - } - }; - } - return new BooleanValueMatcher(false); - } - - return new ValueMatcher() - { - @Override - public boolean matches() - { - int[][] dims = holder.getKey().getDims(); - if (dimIndex >= dims.length || dims[dimIndex] == null) { - return isComparableNullOrEmpty(value); + // filtering on long metrics and __time is supported as well + final Integer metricIndexInt = index.getMetricIndex(dimension); + if (metricIndexInt != null || dimension.equals(Column.TIME_COLUMN_NAME)) { + ValueType type = getTypeForDimension(dimension); + switch (type) { + case LONG: + return Filters.getLongValueMatcher(cursor.makeLongColumnSelector(dimension), originalValue); + default: + return new BooleanValueMatcher(isComparableNullOrEmpty(originalValue)); } - - return Ints.indexOf(dims[dimIndex], id) >= 0; + } else { + return new BooleanValueMatcher(isComparableNullOrEmpty(originalValue)); } - }; + } else { + final DimensionIndexer indexer = dimensionDesc.getIndexer(); + final int dimIndex = dimensionDesc.getIndex(); + return indexer.makeIndexingValueMatcher(originalValue, holder, dimIndex); + } } @Override public ValueMatcher makeValueMatcher(String dimension, final DruidPredicateFactory predicateFactory) - { - ValueType type = getTypeForDimension(dimension); - switch (type) { - case LONG: - return makeLongValueMatcher(dimension, predicateFactory.makeLongPredicate()); - case STRING: - return makeStringValueMatcher(dimension, predicateFactory.makeStringPredicate()); - default: - return new BooleanValueMatcher(predicateFactory.makeStringPredicate().apply(null)); - } - } - - private ValueMatcher makeStringValueMatcher(String dimension, final Predicate predicate) { IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimension); if (dimensionDesc == null) { - return new BooleanValueMatcher(predicate.apply(null)); - } - final int dimIndex = dimensionDesc.getIndex(); - final IncrementalIndex.DimDim dimDim = dimensionDesc.getValues(); - - return new ValueMatcher() - { - @Override - public boolean matches() - { - int[][] dims = holder.getKey().getDims(); - if (dimIndex >= dims.length || dims[dimIndex] == null) { - return predicate.apply(null); - } - - for (int dimVal : dims[dimIndex]) { - if (predicate.apply((String) dimDim.getValue(dimVal))) { - return true; - } + // filtering on long metrics and __time is supported as well + final Integer metricIndexInt = index.getMetricIndex(dimension); + if (metricIndexInt != null || dimension.equals(Column.TIME_COLUMN_NAME)) { + ValueType type = getTypeForDimension(dimension); + switch (type) { + case LONG: + return makeLongValueMatcher(dimension, predicateFactory.makeLongPredicate()); + default: + return new BooleanValueMatcher(predicateFactory.makeStringPredicate().apply(null)); } - return false; + } else { + return new BooleanValueMatcher(predicateFactory.makeStringPredicate().apply(null)); } - }; + } else { + final DimensionIndexer indexer = dimensionDesc.getIndexer(); + final int dimIndex = dimensionDesc.getIndex(); + return indexer.makeIndexingValueMatcher(predicateFactory, holder, dimIndex); + } } + // for long metrics and __time private ValueMatcher makeLongValueMatcher(String dimension, DruidLongPredicate predicate) { return Filters.getLongPredicateMatcher(cursor.makeLongColumnSelector(dimension), predicate); 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 bbb42936192f..9649a6206d35 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -79,7 +79,7 @@ public OffheapIncrementalIndex( this.maxRowCount = maxRowCount; this.bufferPool = bufferPool; - this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator()) + this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) : new PlainFactsHolder(sortFacts); //check that stupid pool gives buffers that can hold at least one row's aggregators @@ -163,12 +163,6 @@ public FactsHolder getFacts() return facts; } - @Override - protected DimDim makeDimDim(String dimension, Object lock) - { - return new OnheapIncrementalIndex.OnHeapDimDim(lock); - } - @Override protected BufferAggregator[] initAggs( AggregatorFactory[] metrics, Supplier rowSupplier, boolean deserializeComplexMetrics @@ -183,7 +177,8 @@ protected BufferAggregator[] initAggs( ColumnSelectorFactory columnSelectorFactory = makeColumnSelectorFactory( agg, rowSupplier, - deserializeComplexMetrics + deserializeComplexMetrics, + getColumnCapabilities() ); selectors.put( @@ -234,7 +229,7 @@ protected Integer addToFacts( for (int i = 0; i < metrics.length; i++) { final AggregatorFactory agg = metrics[i]; getAggs()[i] = agg.factorizeBuffered( - makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics) + makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics, getColumnCapabilities()) ); } rowContainer.set(null); 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 d18a1c527151..3080aa713e26 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -67,7 +67,7 @@ public OnheapIncrementalIndex( super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions); this.maxRowCount = maxRowCount; - this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator()) + this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) : new PlainFactsHolder(sortFacts); } @@ -146,12 +146,6 @@ public FactsHolder getFacts() return facts; } - @Override - protected DimDim makeDimDim(String dimension, Object lock) - { - return new OnHeapDimDim(lock); - } - @Override protected Aggregator[] initAggs( AggregatorFactory[] metrics, Supplier rowSupplier, boolean deserializeComplexMetrics @@ -161,7 +155,7 @@ protected Aggregator[] initAggs( for (AggregatorFactory agg : metrics) { selectors.put( agg.getName(), - new ObjectCachingColumnSelectorFactory(makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics)) + new ObjectCachingColumnSelectorFactory(makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics, getColumnCapabilities())) ); } @@ -335,133 +329,6 @@ public void close() } } - static class OnHeapDimDim> implements DimDim - { - private final Map valueToId = Maps.newHashMap(); - private T minValue = null; - private T maxValue = null; - - private final List idToValue = Lists.newArrayList(); - private final Object lock; - - public OnHeapDimDim(Object lock) - { - this.lock = lock; - } - - public int getId(T value) - { - synchronized (lock) { - final Integer id = valueToId.get(value); - return id == null ? -1 : id; - } - } - - public T getValue(int id) - { - synchronized (lock) { - return idToValue.get(id); - } - } - - public boolean contains(T value) - { - synchronized (lock) { - return valueToId.containsKey(value); - } - } - - public int size() - { - synchronized (lock) { - return valueToId.size(); - } - } - - public int add(T value) - { - synchronized (lock) { - Integer prev = valueToId.get(value); - if (prev != null) { - return prev; - } - final int index = size(); - valueToId.put(value, index); - idToValue.add(value); - minValue = minValue == null || minValue.compareTo(value) > 0 ? value : minValue; - maxValue = maxValue == null || maxValue.compareTo(value) < 0 ? value : maxValue; - return index; - } - } - - @Override - public T getMinValue() - { - return minValue; - } - - @Override - public T getMaxValue() - { - return maxValue; - } - - public OnHeapDimLookup sort() - { - synchronized (lock) { - return new OnHeapDimLookup(idToValue, size()); - } - } - } - - static class OnHeapDimLookup> implements SortedDimLookup - { - private final List sortedVals; - private final int[] idToIndex; - private final int[] indexToId; - - public OnHeapDimLookup(List idToValue, int length) - { - Map sortedMap = Maps.newTreeMap(); - for (int id = 0; id < length; id++) { - sortedMap.put(idToValue.get(id), id); - } - this.sortedVals = Lists.newArrayList(sortedMap.keySet()); - this.idToIndex = new int[length]; - this.indexToId = new int[length]; - int index = 0; - for (Integer id : sortedMap.values()) { - idToIndex[id] = index; - indexToId[index] = id; - index++; - } - } - - @Override - public int size() - { - return sortedVals.size(); - } - - @Override - public int getUnsortedIdFromSortedId(int index) - { - return indexToId[index]; - } - - @Override - public T getValueFromSortedId(int index) - { - return sortedVals.get(index); - } - - @Override - public int getSortedIdFromUnsortedId(int id) - { - return idToIndex[id]; - } - } - // Caches references to selector objects for each column instead of creating a new object each time in order to save heap space. // In general the selectorFactory need not to thread-safe. // here its made thread safe to support the special case of groupBy where the multiple threads can add concurrently to the IncrementalIndex. diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 068c1c430fed..662de75269d5 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -29,6 +29,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; @@ -4785,12 +4786,14 @@ public void testSubqueryWithOuterCardinalityAggregator() .build(); // v1 strategy would throw an exception for this because it calls AggregatorFactory.getRequiredColumns to get - // aggregator for all fields to build the inner query result incremental index. When a field type does not match - // aggregator value type, parse exception occurs. In this case, quality is a string field but getRequiredColumn - // returned a Cardinality aggregator for it, which has type hyperUnique. Since this is a complex type, no converter - // is found for it and NullPointerException occurs when it tries to use the converter. + // aggregator for all fields to build the inner query result incremental index. In this case, quality is a string + // field but getRequiredColumn() returned a Cardinality aggregator for it, which has type hyperUnique. + // The "quality" column is interpreted as a dimension because it appears in the dimension list of the + // MapBasedInputRows from the subquery, but the COMPLEX type from the agg overrides the actual string type. + // COMPLEX is not currently supported as a dimension type, so IAE is thrown. Even if it were, the actual string + // values in the "quality" column could not be interpreted as hyperUniques. if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(NullPointerException.class); + expectedException.expect(IAE.class); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); } else { List expectedResults = Arrays.asList( @@ -4884,11 +4887,16 @@ public void testSubqueryWithOuterJavascriptAggregators() .build(); // v1 strategy would throw an exception for this because it calls AggregatorFactory.getRequiredColumns to get - // aggregator for all fields to build the inner query result incremental index. When a field type does not match - // aggregator value type, parse exception occurs. In this case, market is a string field but getRequiredColumn - // returned a Javascript aggregator for it, which has type float. + // aggregator for all fields to build the inner query result incremental index. In this case, market is a string + // field but getRequiredColumn() returned a Javascript aggregator for it, which has type float. + // The "market" column is interpreted as a dimension because it appears in the dimension list of the + // MapBasedInputRows from the subquery, but the float type from the agg overrides the actual string type. + // Float is not currently supported as a dimension type, so IAE is thrown. Even if it were, a ParseException + // would occur because the "market" column really contains non-numeric values. + // Additionally, the V1 strategy always uses "combining" aggregator factories (meant for merging) on the subquery, + // which does not work for this particular javascript agg. if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(ParseException.class); + expectedException.expect(IAE.class); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); } else { List expectedResults = Arrays.asList( diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTest.java b/processing/src/test/java/io/druid/segment/IndexMergerTest.java index ca7861457639..e86100ab24ea 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTest.java @@ -30,6 +30,7 @@ import com.metamx.collections.bitmap.RoaringBitmapFactory; import com.metamx.common.IAE; import com.metamx.common.ISE; +import com.metamx.common.io.smoosh.SmooshedFileMapper; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.granularity.QueryGranularities; @@ -69,6 +70,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.Map; @@ -1986,6 +1988,50 @@ public void testMismatchedMetricsVarying() throws IOException Assert.assertEquals(ImmutableSet.of("A", "B", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); } + @Test + public void testPersistNullColumnSkipping() throws Exception + { + //check that column d2 is skipped because it only has null values + IncrementalIndex index1 = IncrementalIndexTest.createIndex(new AggregatorFactory[]{ + new LongSumAggregatorFactory("A", "A") + }); + index1.add(new MapBasedInputRow( + 1L, + Lists.newArrayList("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) + )); + + final File tempDir = temporaryFolder.newFolder(); + QueryableIndex index = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + index1, + tempDir, + indexSpec + ) + ) + ); + List expectedColumnNames = Arrays.asList("A", "d1"); + List actualColumnNames = Lists.newArrayList(index.getColumnNames()); + Collections.sort(expectedColumnNames); + Collections.sort(actualColumnNames); + Assert.assertEquals(expectedColumnNames, actualColumnNames); + + SmooshedFileMapper sfm = closer.closeLater(SmooshedFileMapper.load(tempDir)); + List expectedFilenames = Arrays.asList("A", "__time", "d1", "index.drd", "metadata.drd"); + List actualFilenames = new ArrayList<>(sfm.getInternalFilenames()); + Collections.sort(expectedFilenames); + Collections.sort(actualFilenames); + Assert.assertEquals(expectedFilenames, actualFilenames); + } + + private IncrementalIndex getIndexD3() throws Exception { IncrementalIndex toPersist1 = new OnheapIncrementalIndex( diff --git a/processing/src/test/java/io/druid/segment/filter/RowboatTest.java b/processing/src/test/java/io/druid/segment/filter/RowboatTest.java index 402b5f53b681..c5283ac4d034 100644 --- a/processing/src/test/java/io/druid/segment/filter/RowboatTest.java +++ b/processing/src/test/java/io/druid/segment/filter/RowboatTest.java @@ -19,7 +19,9 @@ 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; @@ -28,21 +30,31 @@ */ 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)); + } + return handlers; + } + @Test public void testRowboatCompare() { - Rowboat rb1 = new Rowboat(12345L, new int[][]{new int[]{1}, new int[]{2}}, new Object[]{new Integer(7)}, 5); - Rowboat rb2 = new Rowboat(12345L, new int[][]{new int[]{1}, new int[]{2}}, new Object[]{new Integer(7)}, 5); + 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); + 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[][]{ @@ -62,7 +74,8 @@ public void testBiggerCompare() new int[]{0} }, new Object[]{1.0, 47.0, "someMetric"}, - 0 + 0, + handlers ); Rowboat rb2 = new Rowboat( @@ -84,7 +97,8 @@ public void testBiggerCompare() new int[]{0} }, new Object[]{1.0, 47.0, "someMetric"}, - 0 + 0, + handlers ); Assert.assertNotEquals(0, rb1.compareTo(rb2)); @@ -93,18 +107,22 @@ public void testBiggerCompare() @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).toString() + 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).toString() + new Rowboat(0, null, null, 5, handlers).toString() ); } } diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java index 6d79526e8f13..c931255b3b60 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java @@ -27,6 +27,7 @@ import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.Row; +import io.druid.data.input.impl.DimensionSchema; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.FloatDimensionSchema; import io.druid.data.input.impl.LongDimensionSchema; @@ -75,10 +76,10 @@ public IncrementalIndexTest(IndexCreator IndexCreator) public static Collection constructorFeeder() throws IOException { DimensionsSpec dimensions = new DimensionsSpec( - Arrays.asList( + Arrays.asList( new StringDimensionSchema("string"), - new FloatDimensionSchema("float"), - new LongDimensionSchema("long") + new StringDimensionSchema("float"), + new StringDimensionSchema("long") ), null, null ); AggregatorFactory[] metrics = { @@ -216,8 +217,8 @@ public void testNullDimensionTransform() throws IndexSizeExceededException Row row = index.iterator().next(); Assert.assertEquals(Arrays.asList(new String[]{"", "", "A"}), row.getRaw("string")); - Assert.assertEquals(Arrays.asList(new Float[]{null, null, Float.MAX_VALUE}), row.getRaw("float")); - Assert.assertEquals(Arrays.asList(new Long[]{null, null, Long.MIN_VALUE}), row.getRaw("long")); + Assert.assertEquals(Arrays.asList(new String[]{"", "", String.valueOf(Float.MAX_VALUE)}), row.getRaw("float")); + Assert.assertEquals(Arrays.asList(new String[]{"", "", String.valueOf(Long.MIN_VALUE)}), row.getRaw("long")); } @Test 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 8fc9c0b9ac54..da0117b42074 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -158,7 +158,7 @@ protected Integer addToFacts( for (int i = 0; i < metrics.length; i++) { final AggregatorFactory agg = metrics[i]; aggs[i] = agg.factorize( - makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics) + makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics, null) ); } Integer rowIndex; diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index e34ea81fa6c0..fb6c70f68eea 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -24,12 +24,14 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.common.IAE; import com.metamx.common.ISE; import io.druid.data.input.InputRow; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.QueryableIndex; +import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IndexSizeExceededException; @@ -45,6 +47,7 @@ import java.util.Iterator; import java.util.LinkedHashSet; import java.util.List; +import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; @@ -262,16 +265,20 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) FireHydrant lastHydrant = hydrants.get(numHydrants - 1); newCount = lastHydrant.getCount() + 1; if (!indexSchema.getDimensionsSpec().hasCustomDimensions()) { + Map oldCapabilities; if (lastHydrant.hasSwapped()) { + oldCapabilities = Maps.newHashMap(); QueryableIndex oldIndex = lastHydrant.getSegment().asQueryableIndex(); for (String dim : oldIndex.getAvailableDimensions()) { dimOrder.add(dim); + oldCapabilities.put(dim, (ColumnCapabilitiesImpl) oldIndex.getColumn(dim).getCapabilities()); } } else { IncrementalIndex oldIndex = lastHydrant.getIndex(); dimOrder.addAll(oldIndex.getDimensionOrder()); + oldCapabilities = oldIndex.getColumnCapabilities(); } - newIndex.loadDimensionIterable(dimOrder); + newIndex.loadDimensionIterable(dimOrder, oldCapabilities); } } currHydrant = new FireHydrant(newIndex, newCount, getSegment().getIdentifier());