mirror of https://github.com/apache/druid.git
Add dimension handling interface for ingestion and segment creation (#3217)
* Add dimension handling interface for ingestion and segment creation * update javadocs for DimensionHandler/DimensionIndexer * Move IndexIO row validation into DimensionHandler * Fix null column skipping in mergerV9 * Add deprecation note for 'numeric_dims' filename pattern in IndexIO v8->v9 conversion * Fix java7 test failure
This commit is contained in:
parent
1a5042ca26
commit
df766b2bbd
|
@ -97,7 +97,7 @@ public class IndexIngestionBenchmark
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Setup(Level.Iteration)
|
@Setup(Level.Invocation)
|
||||||
public void setup2() throws IOException
|
public void setup2() throws IOException
|
||||||
{
|
{
|
||||||
incIndex = makeIncIndex();
|
incIndex = makeIncIndex();
|
||||||
|
|
|
@ -1523,7 +1523,7 @@ public class KafkaIndexTaskTest
|
||||||
);
|
);
|
||||||
IndexIO indexIO = new TestUtils().getTestIndexIO();
|
IndexIO indexIO = new TestUtils().getTestIndexIO();
|
||||||
QueryableIndex index = indexIO.loadIndex(outputLocation);
|
QueryableIndex index = indexIO.loadIndex(outputLocation);
|
||||||
DictionaryEncodedColumn dim1 = index.getColumn("dim1").getDictionaryEncoding();
|
DictionaryEncodedColumn<String> dim1 = index.getColumn("dim1").getDictionaryEncoding();
|
||||||
List<String> values = Lists.newArrayList();
|
List<String> values = Lists.newArrayList();
|
||||||
for (int i = 0; i < dim1.length(); i++) {
|
for (int i = 0; i < dim1.length(); i++) {
|
||||||
int id = dim1.getSingleValueRow(i);
|
int id = dim1.getSingleValueRow(i);
|
||||||
|
|
|
@ -46,6 +46,8 @@ import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.segment.BaseProgressIndicator;
|
import io.druid.segment.BaseProgressIndicator;
|
||||||
import io.druid.segment.ProgressIndicator;
|
import io.druid.segment.ProgressIndicator;
|
||||||
import io.druid.segment.QueryableIndex;
|
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.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
||||||
|
@ -77,6 +79,7 @@ import java.nio.ByteBuffer;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.LinkedHashSet;
|
import java.util.LinkedHashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.BlockingQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
@ -216,7 +219,8 @@ public class IndexGeneratorJob implements Jobby
|
||||||
Bucket theBucket,
|
Bucket theBucket,
|
||||||
AggregatorFactory[] aggs,
|
AggregatorFactory[] aggs,
|
||||||
HadoopDruidIndexerConfig config,
|
HadoopDruidIndexerConfig config,
|
||||||
Iterable<String> oldDimOrder
|
Iterable<String> oldDimOrder,
|
||||||
|
Map<String, ColumnCapabilitiesImpl> oldCapabilities
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final HadoopTuningConfig tuningConfig = config.getSchema().getTuningConfig();
|
final HadoopTuningConfig tuningConfig = config.getSchema().getTuningConfig();
|
||||||
|
@ -236,7 +240,7 @@ public class IndexGeneratorJob implements Jobby
|
||||||
);
|
);
|
||||||
|
|
||||||
if (oldDimOrder != null && !indexSchema.getDimensionsSpec().hasCustomDimensions()) {
|
if (oldDimOrder != null && !indexSchema.getDimensionsSpec().hasCustomDimensions()) {
|
||||||
newIndex.loadDimensionIterable(oldDimOrder);
|
newIndex.loadDimensionIterable(oldDimOrder, oldCapabilities);
|
||||||
}
|
}
|
||||||
|
|
||||||
return newIndex;
|
return newIndex;
|
||||||
|
@ -340,7 +344,7 @@ public class IndexGeneratorJob implements Jobby
|
||||||
LinkedHashSet<String> dimOrder = Sets.newLinkedHashSet();
|
LinkedHashSet<String> dimOrder = Sets.newLinkedHashSet();
|
||||||
SortableBytes keyBytes = SortableBytes.fromBytesWritable(key);
|
SortableBytes keyBytes = SortableBytes.fromBytesWritable(key);
|
||||||
Bucket bucket = Bucket.fromGroupKey(keyBytes.getGroupKey()).lhs;
|
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));
|
index.add(InputRowSerde.fromBytes(first.getBytes(), aggregators));
|
||||||
|
|
||||||
while (iter.hasNext()) {
|
while (iter.hasNext()) {
|
||||||
|
@ -351,7 +355,7 @@ public class IndexGeneratorJob implements Jobby
|
||||||
dimOrder.addAll(index.getDimensionOrder());
|
dimOrder.addAll(index.getDimensionOrder());
|
||||||
log.info("current index full due to [%s]. creating new index.", index.getOutOfRowsReason());
|
log.info("current index full due to [%s]. creating new index.", index.getOutOfRowsReason());
|
||||||
flushIndexToContextAndClose(key, index, context);
|
flushIndexToContextAndClose(key, index, context);
|
||||||
index = makeIncrementalIndex(bucket, combiningAggs, config, dimOrder);
|
index = makeIncrementalIndex(bucket, combiningAggs, config, dimOrder, index.getColumnCapabilities());
|
||||||
}
|
}
|
||||||
|
|
||||||
index.add(value);
|
index.add(value);
|
||||||
|
@ -557,6 +561,7 @@ public class IndexGeneratorJob implements Jobby
|
||||||
bucket,
|
bucket,
|
||||||
combiningAggs,
|
combiningAggs,
|
||||||
config,
|
config,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
try {
|
try {
|
||||||
|
@ -654,7 +659,8 @@ public class IndexGeneratorJob implements Jobby
|
||||||
bucket,
|
bucket,
|
||||||
combiningAggs,
|
combiningAggs,
|
||||||
config,
|
config,
|
||||||
allDimensionNames
|
allDimensionNames,
|
||||||
|
persistIndex.getColumnCapabilities()
|
||||||
);
|
);
|
||||||
startTime = System.currentTimeMillis();
|
startTime = System.currentTimeMillis();
|
||||||
++indexCount;
|
++indexCount;
|
||||||
|
|
|
@ -89,7 +89,8 @@ public class InputRowSerde
|
||||||
IncrementalIndex.makeColumnSelectorFactory(
|
IncrementalIndex.makeColumnSelectorFactory(
|
||||||
aggFactory,
|
aggFactory,
|
||||||
supplier,
|
supplier,
|
||||||
true
|
true,
|
||||||
|
null
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -60,7 +60,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
||||||
if (columnDesc == null || !columnDesc.getCapabilities().isDictionaryEncoded()) {
|
if (columnDesc == null || !columnDesc.getCapabilities().isDictionaryEncoded()) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
final DictionaryEncodedColumn column = columnDesc.getDictionaryEncoding();
|
final DictionaryEncodedColumn<String> column = columnDesc.getDictionaryEncoding();
|
||||||
return new Indexed<String>()
|
return new Indexed<String>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -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 <EncodedType> class of the encoded values
|
||||||
|
* @param <ActualType> class of the actual values
|
||||||
|
*/
|
||||||
|
public interface DimensionHandler<EncodedType extends Comparable<EncodedType>, EncodedTypeArray, ActualType extends Comparable<ActualType>>
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* 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<EncodedType, EncodedTypeArray, ActualType> 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<EncodedTypeArray> 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<EncodedTypeArray> 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<ActualType> lhsEncodings,
|
||||||
|
Indexed<ActualType> 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);
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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 <EncodedType> class of the encoded values
|
||||||
|
* @param <ActualType> class of the actual values
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public interface DimensionIndexer<EncodedType extends Comparable<EncodedType>, EncodedTypeArray, ActualType extends Comparable<ActualType>>
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* 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<ActualType> 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);
|
||||||
|
}
|
|
@ -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<EncodedTypedArray>
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* 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<IndexableAdapter> 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<IntBuffer> 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();
|
||||||
|
}
|
|
@ -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<EncodedTypeArray> extends DimensionMergerV9<EncodedTypeArray>
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* 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<FileOutputStream> spatialOut
|
||||||
|
) throws IOException;
|
||||||
|
}
|
|
@ -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<EncodedTypeArray> extends DimensionMerger<EncodedTypeArray>
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* 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();
|
||||||
|
}
|
|
@ -52,6 +52,7 @@ import com.metamx.emitter.EmittingLogger;
|
||||||
import io.druid.common.utils.SerializerUtils;
|
import io.druid.common.utils.SerializerUtils;
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
import io.druid.segment.column.ColumnBuilder;
|
import io.druid.segment.column.ColumnBuilder;
|
||||||
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.column.ColumnConfig;
|
import io.druid.segment.column.ColumnConfig;
|
||||||
import io.druid.segment.column.ColumnDescriptor;
|
import io.druid.segment.column.ColumnDescriptor;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
|
@ -66,6 +67,7 @@ import io.druid.segment.data.GenericIndexed;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.data.IndexedIterable;
|
import io.druid.segment.data.IndexedIterable;
|
||||||
|
import io.druid.segment.data.IndexedLongs;
|
||||||
import io.druid.segment.data.IndexedMultivalue;
|
import io.druid.segment.data.IndexedMultivalue;
|
||||||
import io.druid.segment.data.IndexedRTree;
|
import io.druid.segment.data.IndexedRTree;
|
||||||
import io.druid.segment.data.VSizeIndexed;
|
import io.druid.segment.data.VSizeIndexed;
|
||||||
|
@ -87,6 +89,7 @@ import java.io.File;
|
||||||
import java.io.FileInputStream;
|
import java.io.FileInputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
|
import java.lang.reflect.Array;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.nio.ByteOrder;
|
import java.nio.ByteOrder;
|
||||||
import java.util.AbstractList;
|
import java.util.AbstractList;
|
||||||
|
@ -173,6 +176,8 @@ public class IndexIO
|
||||||
throw new SegmentValidationException("Metric names differ. Expected [%s] found [%s]", metNames1, metNames2);
|
throw new SegmentValidationException("Metric names differ. Expected [%s] found [%s]", metNames1, metNames2);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
final Map<String, DimensionHandler> dimHandlers = adapter1.getDimensionHandlers();
|
||||||
|
|
||||||
final Iterator<Rowboat> it1 = adapter1.getRows().iterator();
|
final Iterator<Rowboat> it1 = adapter1.getRows().iterator();
|
||||||
final Iterator<Rowboat> it2 = adapter2.getRows().iterator();
|
final Iterator<Rowboat> it2 = adapter2.getRows().iterator();
|
||||||
long row = 0L;
|
long row = 0L;
|
||||||
|
@ -188,7 +193,7 @@ public class IndexIO
|
||||||
}
|
}
|
||||||
if (rb1.compareTo(rb2) != 0) {
|
if (rb1.compareTo(rb2) != 0) {
|
||||||
try {
|
try {
|
||||||
validateRowValues(rb1, adapter1, rb2, adapter2);
|
validateRowValues(dimHandlers, rb1, adapter1, rb2, adapter2);
|
||||||
}
|
}
|
||||||
catch (SegmentValidationException ex) {
|
catch (SegmentValidationException ex) {
|
||||||
throw new SegmentValidationException(ex, "Validation failure on row %d: [%s] vs [%s]", row, rb1, rb2);
|
throw new SegmentValidationException(ex, "Validation failure on row %d: [%s] vs [%s]", row, rb1, rb2);
|
||||||
|
@ -307,6 +312,7 @@ public class IndexIO
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void validateRowValues(
|
public static void validateRowValues(
|
||||||
|
Map<String, DimensionHandler> dimHandlers,
|
||||||
Rowboat rb1,
|
Rowboat rb1,
|
||||||
IndexableAdapter adapter1,
|
IndexableAdapter adapter1,
|
||||||
Rowboat rb2,
|
Rowboat rb2,
|
||||||
|
@ -319,8 +325,8 @@ public class IndexIO
|
||||||
rb1.getTimestamp(), rb2.getTimestamp()
|
rb1.getTimestamp(), rb2.getTimestamp()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
final int[][] dims1 = rb1.getDims();
|
final Object[] dims1 = rb1.getDims();
|
||||||
final int[][] dims2 = rb2.getDims();
|
final Object[] dims2 = rb2.getDims();
|
||||||
if (dims1.length != dims2.length) {
|
if (dims1.length != dims2.length) {
|
||||||
throw new SegmentValidationException(
|
throw new SegmentValidationException(
|
||||||
"Dim lengths not equal %s vs %s",
|
"Dim lengths not equal %s vs %s",
|
||||||
|
@ -331,90 +337,31 @@ public class IndexIO
|
||||||
final Indexed<String> dim1Names = adapter1.getDimensionNames();
|
final Indexed<String> dim1Names = adapter1.getDimensionNames();
|
||||||
final Indexed<String> dim2Names = adapter2.getDimensionNames();
|
final Indexed<String> dim2Names = adapter2.getDimensionNames();
|
||||||
for (int i = 0; i < dims1.length; ++i) {
|
for (int i = 0; i < dims1.length; ++i) {
|
||||||
final int[] dim1Vals = dims1[i];
|
final Object dim1Vals = dims1[i];
|
||||||
final int[] dim2Vals = dims2[i];
|
final Object dim2Vals = dims2[i];
|
||||||
final String dim1Name = dim1Names.get(i);
|
final String dim1Name = dim1Names.get(i);
|
||||||
final String dim2Name = dim2Names.get(i);
|
final String dim2Name = dim2Names.get(i);
|
||||||
final Indexed<String> dim1ValNames = adapter1.getDimValueLookup(dim1Name);
|
|
||||||
final Indexed<String> dim2ValNames = adapter2.getDimValueLookup(dim2Name);
|
|
||||||
|
|
||||||
if (dim1Vals == null || dim2Vals == null) {
|
ColumnCapabilities capabilities1 = adapter1.getCapabilities(dim1Name);
|
||||||
if (dim1Vals != dim2Vals) {
|
ColumnCapabilities capabilities2 = adapter2.getCapabilities(dim2Name);
|
||||||
throw new SegmentValidationException(
|
ValueType dim1Type = capabilities1.getType();
|
||||||
"Expected nulls, found %s and %s",
|
ValueType dim2Type = capabilities2.getType();
|
||||||
Arrays.toString(dim1Vals),
|
if (dim1Type != dim2Type) {
|
||||||
Arrays.toString(dim2Vals)
|
throw new SegmentValidationException(
|
||||||
);
|
"Dim [%s] types not equal. Expected %d found %d",
|
||||||
} else {
|
dim1Name,
|
||||||
continue;
|
dim1Type,
|
||||||
}
|
dim2Type
|
||||||
}
|
);
|
||||||
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) {
|
DimensionHandler dimHandler = dimHandlers.get(dim1Name);
|
||||||
final int dIdex1 = dim1Vals.length <= j ? -1 : dim1Vals[j];
|
dimHandler.validateSortedEncodedArrays(
|
||||||
final int dIdex2 = dim2Vals.length <= j ? -1 : dim2Vals[j];
|
dim1Vals,
|
||||||
|
dim2Vals,
|
||||||
if (dIdex1 == dIdex2) {
|
adapter1.getDimValueLookup(dim1Name),
|
||||||
continue;
|
adapter2.getDimValueLookup(dim2Name)
|
||||||
}
|
);
|
||||||
|
|
||||||
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
|
|
||||||
);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -748,7 +695,10 @@ public class IndexIO
|
||||||
channel.write(ByteBuffer.wrap(specBytes));
|
channel.write(ByteBuffer.wrap(specBytes));
|
||||||
serdeficator.write(channel);
|
serdeficator.write(channel);
|
||||||
channel.close();
|
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))) {
|
if (!filename.endsWith(String.format("%s.drd", BYTE_ORDER))) {
|
||||||
skippedFiles.add(filename);
|
skippedFiles.add(filename);
|
||||||
continue;
|
continue;
|
||||||
|
@ -1101,6 +1051,11 @@ public class IndexIO
|
||||||
return new File(dir, String.format("dim_%s.drd", 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)
|
public static File makeTimeFile(File dir, ByteOrder order)
|
||||||
{
|
{
|
||||||
return new File(dir, String.format("time_%s.drd", order));
|
return new File(dir, String.format("time_%s.drd", order));
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.google.common.base.Function;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Splitter;
|
import com.google.common.base.Splitter;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.collect.FluentIterable;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Iterators;
|
import com.google.common.collect.Iterators;
|
||||||
|
@ -34,7 +33,6 @@ import com.google.common.collect.Ordering;
|
||||||
import com.google.common.collect.PeekingIterator;
|
import com.google.common.collect.PeekingIterator;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.google.common.io.ByteSink;
|
import com.google.common.io.ByteSink;
|
||||||
import com.google.common.io.ByteStreams;
|
|
||||||
import com.google.common.io.Closer;
|
import com.google.common.io.Closer;
|
||||||
import com.google.common.io.FileWriteMode;
|
import com.google.common.io.FileWriteMode;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
|
@ -42,13 +40,6 @@ import com.google.common.io.OutputSupplier;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
import com.google.inject.Inject;
|
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.IAE;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.Pair;
|
import com.metamx.common.Pair;
|
||||||
|
@ -67,20 +58,16 @@ import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
import io.druid.segment.data.BitmapSerdeFactory;
|
import io.druid.segment.data.BitmapSerdeFactory;
|
||||||
import io.druid.segment.data.ByteBufferWriter;
|
|
||||||
import io.druid.segment.data.CompressedObjectStrategy;
|
import io.druid.segment.data.CompressedObjectStrategy;
|
||||||
import io.druid.segment.data.CompressionFactory;
|
import io.druid.segment.data.CompressionFactory;
|
||||||
import io.druid.segment.data.LongSupplierSerializer;
|
import io.druid.segment.data.LongSupplierSerializer;
|
||||||
import io.druid.segment.data.GenericIndexed;
|
import io.druid.segment.data.GenericIndexed;
|
||||||
import io.druid.segment.data.GenericIndexedWriter;
|
|
||||||
import io.druid.segment.data.IOPeon;
|
import io.druid.segment.data.IOPeon;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.data.IndexedIterable;
|
import io.druid.segment.data.IndexedIterable;
|
||||||
import io.druid.segment.data.IndexedRTree;
|
|
||||||
import io.druid.segment.data.ListIndexed;
|
import io.druid.segment.data.ListIndexed;
|
||||||
import io.druid.segment.data.TmpFileIOPeon;
|
import io.druid.segment.data.TmpFileIOPeon;
|
||||||
import io.druid.segment.data.VSizeIndexedWriter;
|
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexAdapter;
|
import io.druid.segment.incremental.IncrementalIndexAdapter;
|
||||||
import io.druid.segment.serde.ComplexMetricColumnSerializer;
|
import io.druid.segment.serde.ComplexMetricColumnSerializer;
|
||||||
|
@ -97,7 +84,6 @@ import java.io.FileOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.nio.IntBuffer;
|
import java.nio.IntBuffer;
|
||||||
import java.nio.MappedByteBuffer;
|
|
||||||
import java.nio.channels.FileChannel;
|
import java.nio.channels.FileChannel;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
@ -611,6 +597,7 @@ public class IndexMerger
|
||||||
final Map<String, ValueType> valueTypes = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
|
final Map<String, ValueType> valueTypes = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
|
||||||
final Map<String, String> metricTypeNames = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
|
final Map<String, String> metricTypeNames = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
|
||||||
final Map<String, ColumnCapabilitiesImpl> columnCapabilities = Maps.newHashMap();
|
final Map<String, ColumnCapabilitiesImpl> columnCapabilities = Maps.newHashMap();
|
||||||
|
final List<ColumnCapabilitiesImpl> dimCapabilities = new ArrayList<>();
|
||||||
|
|
||||||
for (IndexableAdapter adapter : indexes) {
|
for (IndexableAdapter adapter : indexes) {
|
||||||
for (String dimension : adapter.getDimensionNames()) {
|
for (String dimension : adapter.getDimensionNames()) {
|
||||||
|
@ -618,7 +605,6 @@ public class IndexMerger
|
||||||
ColumnCapabilities capabilities = adapter.getCapabilities(dimension);
|
ColumnCapabilities capabilities = adapter.getCapabilities(dimension);
|
||||||
if (mergedCapabilities == null) {
|
if (mergedCapabilities == null) {
|
||||||
mergedCapabilities = new ColumnCapabilitiesImpl();
|
mergedCapabilities = new ColumnCapabilitiesImpl();
|
||||||
mergedCapabilities.setType(ValueType.STRING);
|
|
||||||
}
|
}
|
||||||
columnCapabilities.put(dimension, mergedCapabilities.merge(capabilities));
|
columnCapabilities.put(dimension, mergedCapabilities.merge(capabilities));
|
||||||
}
|
}
|
||||||
|
@ -635,6 +621,10 @@ public class IndexMerger
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
for (String dimension : mergedDimensions) {
|
||||||
|
dimCapabilities.add(columnCapabilities.get(dimension));
|
||||||
|
}
|
||||||
|
|
||||||
Closer closer = Closer.create();
|
Closer closer = Closer.create();
|
||||||
final Interval dataInterval;
|
final Interval dataInterval;
|
||||||
final File v8OutDir = new File(outDir, "v8-tmp");
|
final File v8OutDir = new File(outDir, "v8-tmp");
|
||||||
|
@ -689,93 +679,22 @@ public class IndexMerger
|
||||||
startTime = System.currentTimeMillis();
|
startTime = System.currentTimeMillis();
|
||||||
|
|
||||||
final ArrayList<FileOutputSupplier> dimOuts = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
final ArrayList<FileOutputSupplier> dimOuts = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
||||||
final Map<String, Integer> dimensionCardinalities = Maps.newHashMap();
|
final DimensionHandler[] handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities);
|
||||||
final ArrayList<Map<String, IntBuffer>> dimConversions = Lists.newArrayListWithCapacity(indexes.size());
|
final List<DimensionMerger> mergers = new ArrayList<>();
|
||||||
final ArrayList<Boolean> convertMissingDimsFlags = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
for (int i = 0; i < mergedDimensions.size(); i++) {
|
||||||
final ArrayList<MutableBitmap> nullRowsList = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
DimensionMergerLegacy merger = handlers[i].makeLegacyMerger(
|
||||||
final ArrayList<Boolean> dimHasNullFlags = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
indexSpec,
|
||||||
|
v8OutDir,
|
||||||
for (int i = 0; i < indexes.size(); ++i) {
|
ioPeon,
|
||||||
dimConversions.add(Maps.<String, IntBuffer>newHashMap());
|
dimCapabilities.get(i),
|
||||||
}
|
progress
|
||||||
|
|
||||||
for (String dimension : mergedDimensions) {
|
|
||||||
nullRowsList.add(indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap());
|
|
||||||
|
|
||||||
final GenericIndexedWriter<String> writer = new GenericIndexedWriter<String>(
|
|
||||||
ioPeon, dimension, GenericIndexed.STRING_STRATEGY
|
|
||||||
);
|
);
|
||||||
writer.open();
|
mergers.add(merger);
|
||||||
|
merger.writeMergedValueMetadata(indexes);
|
||||||
|
|
||||||
boolean dimHasNull = false;
|
FileOutputSupplier dimOut = new FileOutputSupplier(IndexIO.makeDimFile(v8OutDir, mergedDimensions.get(i)), true);
|
||||||
boolean dimHasValues = false;
|
merger.writeValueMetadataToFile(dimOut);
|
||||||
boolean dimAbsentFromSomeIndex = false;
|
|
||||||
|
|
||||||
int numMergeIndex = 0;
|
|
||||||
Indexed<String> dimValueLookup = null;
|
|
||||||
Indexed<String>[] dimValueLookups = new Indexed[indexes.size() + 1];
|
|
||||||
for (int i = 0; i < indexes.size(); i++) {
|
|
||||||
Indexed<String> 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());
|
|
||||||
}
|
|
||||||
|
|
||||||
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);
|
|
||||||
dimOuts.add(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);
|
log.info("outDir[%s] completed dim conversions in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime);
|
||||||
|
|
||||||
|
@ -787,9 +706,10 @@ public class IndexMerger
|
||||||
indexes,
|
indexes,
|
||||||
mergedDimensions,
|
mergedDimensions,
|
||||||
mergedMetrics,
|
mergedMetrics,
|
||||||
dimConversions,
|
rowMergerFn,
|
||||||
convertMissingDimsFlags,
|
dimCapabilities,
|
||||||
rowMergerFn
|
handlers,
|
||||||
|
mergers
|
||||||
);
|
);
|
||||||
|
|
||||||
LongSupplierSerializer timeWriter = CompressionFactory.getLongSerializer(
|
LongSupplierSerializer timeWriter = CompressionFactory.getLongSerializer(
|
||||||
|
@ -799,13 +719,6 @@ public class IndexMerger
|
||||||
|
|
||||||
timeWriter.open();
|
timeWriter.open();
|
||||||
|
|
||||||
ArrayList<VSizeIndexedWriter> forwardDimWriters = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
|
||||||
for (String dimension : mergedDimensions) {
|
|
||||||
VSizeIndexedWriter writer = new VSizeIndexedWriter(ioPeon, dimension, dimensionCardinalities.get(dimension));
|
|
||||||
writer.open();
|
|
||||||
forwardDimWriters.add(writer);
|
|
||||||
}
|
|
||||||
|
|
||||||
ArrayList<MetricColumnSerializer> metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size());
|
ArrayList<MetricColumnSerializer> metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size());
|
||||||
final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression();
|
final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression();
|
||||||
final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding();
|
final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding();
|
||||||
|
@ -855,20 +768,9 @@ public class IndexMerger
|
||||||
metWriters.get(i).serialize(metrics[i]);
|
metWriters.get(i).serialize(metrics[i]);
|
||||||
}
|
}
|
||||||
|
|
||||||
int[][] dims = theRow.getDims();
|
Object[] dims = theRow.getDims();
|
||||||
for (int i = 0; i < dims.length; ++i) {
|
for (int i = 0; i < dims.length; ++i) {
|
||||||
List<Integer> listToWrite = (i >= dims.length || dims[i] == null)
|
mergers.get(i).processMergedRow(dims[i]);
|
||||||
? 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);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for (Map.Entry<Integer, TreeSet<Integer>> comprisedRow : theRow.getComprisedRows().entrySet()) {
|
for (Map.Entry<Integer, TreeSet<Integer>> comprisedRow : theRow.getComprisedRows().entrySet()) {
|
||||||
|
@ -900,16 +802,10 @@ public class IndexMerger
|
||||||
timeWriter.closeAndConsolidate(out);
|
timeWriter.closeAndConsolidate(out);
|
||||||
IndexIO.checkFileSize(timeFile);
|
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) {
|
for (MetricColumnSerializer metWriter : metWriters) {
|
||||||
metWriter.close();
|
metWriter.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
ioPeon.cleanup();
|
|
||||||
log.info(
|
log.info(
|
||||||
"outDir[%s] completed walk through of %,d rows in %,d millis.",
|
"outDir[%s] completed walk through of %,d rows in %,d millis.",
|
||||||
v8OutDir,
|
v8OutDir,
|
||||||
|
@ -917,7 +813,7 @@ public class IndexMerger
|
||||||
System.currentTimeMillis() - startTime
|
System.currentTimeMillis() - startTime
|
||||||
);
|
);
|
||||||
|
|
||||||
/************ Create Inverted Indexes *************/
|
/************ Create Inverted Indexes and Finalize Columns *************/
|
||||||
startTime = System.currentTimeMillis();
|
startTime = System.currentTimeMillis();
|
||||||
|
|
||||||
final File invertedFile = new File(v8OutDir, "inverted.drd");
|
final File invertedFile = new File(v8OutDir, "inverted.drd");
|
||||||
|
@ -928,119 +824,35 @@ public class IndexMerger
|
||||||
Files.touch(geoFile);
|
Files.touch(geoFile);
|
||||||
OutputSupplier<FileOutputStream> spatialOut = Files.newOutputStreamSupplier(geoFile, true);
|
OutputSupplier<FileOutputStream> spatialOut = Files.newOutputStreamSupplier(geoFile, true);
|
||||||
|
|
||||||
for (int i = 0; i < mergedDimensions.size(); ++i) {
|
for (int i = 0; i < mergedDimensions.size(); i++) {
|
||||||
long dimStartTime = System.currentTimeMillis();
|
DimensionMergerLegacy legacyMerger = (DimensionMergerLegacy) mergers.get(i);
|
||||||
String dimension = mergedDimensions.get(i);
|
legacyMerger.writeIndexes(rowNumConversions, closer);
|
||||||
|
legacyMerger.writeIndexesToFiles(out, spatialOut);
|
||||||
File dimOutFile = dimOuts.get(i).getFile();
|
legacyMerger.writeRowValuesToFile(dimOuts.get(i));
|
||||||
final MappedByteBuffer dimValsMapped = Files.map(dimOutFile);
|
|
||||||
closer.register(new Closeable()
|
|
||||||
{
|
|
||||||
@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<String> dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY);
|
|
||||||
log.info("Starting dimension[%s] with cardinality[%,d]", dimension, dimVals.size());
|
|
||||||
|
|
||||||
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
|
|
||||||
GenericIndexedWriter<ImmutableBitmap> writer = new GenericIndexedWriter<>(
|
|
||||||
ioPeon, dimension, bitmapSerdeFactory.getObjectStrategy()
|
|
||||||
);
|
|
||||||
writer.open();
|
|
||||||
|
|
||||||
boolean isSpatialDim = columnCapabilities.get(dimension).hasSpatialIndexes();
|
|
||||||
ByteBufferWriter<ImmutableRTree> spatialWriter = null;
|
|
||||||
RTree tree = null;
|
|
||||||
IOPeon spatialIoPeon = new TmpFileIOPeon();
|
|
||||||
if (isSpatialDim) {
|
|
||||||
BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory();
|
|
||||||
spatialWriter = new ByteBufferWriter<ImmutableRTree>(
|
|
||||||
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<Iterable<Integer>> 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.<Integer>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<String> 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);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
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 and wrote dimensions in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime);
|
||||||
|
|
||||||
log.info("outDir[%s] completed inverted.drd in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime);
|
final Function<String, String> dimFilenameFunction = new Function<String, String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String apply(@Nullable String input)
|
||||||
|
{
|
||||||
|
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);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
final ArrayList<String> expectedFiles = Lists.newArrayList(
|
final ArrayList<String> expectedFiles = Lists.newArrayList(
|
||||||
Iterables.concat(
|
Iterables.concat(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"index.drd", "inverted.drd", "spatial.drd", String.format("time_%s.drd", IndexIO.BYTE_ORDER)
|
"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(
|
Iterables.transform(
|
||||||
mergedMetrics, GuavaUtils.formatFunction(String.format("met_%%s_%s.drd", IndexIO.BYTE_ORDER))
|
mergedMetrics, GuavaUtils.formatFunction(String.format("met_%%s_%s.drd", IndexIO.BYTE_ORDER))
|
||||||
)
|
)
|
||||||
|
@ -1095,13 +907,25 @@ public class IndexMerger
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected DimensionHandler[] makeDimensionHandlers(final List<String> mergedDimensions, final List<ColumnCapabilitiesImpl> 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<Rowboat> makeRowIterable(
|
protected Iterable<Rowboat> makeRowIterable(
|
||||||
List<IndexableAdapter> indexes,
|
List<IndexableAdapter> indexes,
|
||||||
final List<String> mergedDimensions,
|
final List<String> mergedDimensions,
|
||||||
final List<String> mergedMetrics,
|
final List<String> mergedMetrics,
|
||||||
ArrayList<Map<String, IntBuffer>> dimConversions,
|
Function<ArrayList<Iterable<Rowboat>>, Iterable<Rowboat>> rowMergerFn,
|
||||||
ArrayList<Boolean> convertMissingDimsFlags,
|
final List<ColumnCapabilitiesImpl> dimCapabilities,
|
||||||
Function<ArrayList<Iterable<Rowboat>>, Iterable<Rowboat>> rowMergerFn
|
final DimensionHandler[] handlers,
|
||||||
|
final List<DimensionMerger> mergers
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
ArrayList<Iterable<Rowboat>> boats = Lists.newArrayListWithCapacity(indexes.size());
|
ArrayList<Iterable<Rowboat>> boats = Lists.newArrayListWithCapacity(indexes.size());
|
||||||
|
@ -1109,8 +933,8 @@ public class IndexMerger
|
||||||
for (int i = 0; i < indexes.size(); ++i) {
|
for (int i = 0; i < indexes.size(); ++i) {
|
||||||
final IndexableAdapter adapter = indexes.get(i);
|
final IndexableAdapter adapter = indexes.get(i);
|
||||||
|
|
||||||
final int[] dimLookup = toLookupMap(adapter.getDimensionNames(), mergedDimensions);
|
final int[] dimLookup = getColumnIndexReorderingMap(adapter.getDimensionNames(), mergedDimensions);
|
||||||
final int[] metricLookup = toLookupMap(adapter.getMetricNames(), mergedMetrics);
|
final int[] metricLookup = getColumnIndexReorderingMap(adapter.getMetricNames(), mergedMetrics);
|
||||||
|
|
||||||
Iterable<Rowboat> target = indexes.get(i).getRows();
|
Iterable<Rowboat> target = indexes.get(i).getRows();
|
||||||
if (dimLookup != null || metricLookup != null) {
|
if (dimLookup != null || metricLookup != null) {
|
||||||
|
@ -1122,14 +946,18 @@ public class IndexMerger
|
||||||
@Override
|
@Override
|
||||||
public Rowboat apply(Rowboat input)
|
public Rowboat apply(Rowboat input)
|
||||||
{
|
{
|
||||||
int[][] newDims = input.getDims();
|
Object[] newDims;
|
||||||
if (dimLookup != null) {
|
if (dimLookup != null) {
|
||||||
newDims = new int[mergedDimensions.size()][];
|
newDims = new Object[mergedDimensions.size()];
|
||||||
int j = 0;
|
int j = 0;
|
||||||
for (int[] dim : input.getDims()) {
|
for (Object dim : input.getDims()) {
|
||||||
newDims[dimLookup[j]] = dim;
|
newDims[dimLookup[j]] = dim;
|
||||||
j++;
|
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();
|
Object[] newMetrics = input.getMetrics();
|
||||||
|
@ -1146,7 +974,8 @@ public class IndexMerger
|
||||||
input.getTimestamp(),
|
input.getTimestamp(),
|
||||||
newDims,
|
newDims,
|
||||||
newMetrics,
|
newMetrics,
|
||||||
input.getRowNum()
|
input.getRowNum(),
|
||||||
|
handlers
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1154,7 +983,7 @@ public class IndexMerger
|
||||||
}
|
}
|
||||||
boats.add(
|
boats.add(
|
||||||
new MMappedIndexRowIterable(
|
new MMappedIndexRowIterable(
|
||||||
target, mergedDimensions, dimConversions.get(i), i, convertMissingDimsFlags
|
target, mergedDimensions, i, dimCapabilities, mergers
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -1162,14 +991,16 @@ public class IndexMerger
|
||||||
return rowMergerFn.apply(boats);
|
return rowMergerFn.apply(boats);
|
||||||
}
|
}
|
||||||
|
|
||||||
private int[] toLookupMap(Indexed<String> indexed, List<String> 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<String> adapterColumnNames, List<String> mergedColumnNames)
|
||||||
{
|
{
|
||||||
if (isSame(indexed, values)) {
|
if (isSame(adapterColumnNames, mergedColumnNames)) {
|
||||||
return null; // no need to convert
|
return null; // no need to convert if column lists are identical
|
||||||
}
|
}
|
||||||
int[] dimLookup = new int[values.size()];
|
int[] dimLookup = new int[mergedColumnNames.size()];
|
||||||
for (int i = 0; i < indexed.size(); i++) {
|
for (int i = 0; i < adapterColumnNames.size(); i++) {
|
||||||
dimLookup[i] = values.indexOf(indexed.get(i));
|
dimLookup[i] = mergedColumnNames.indexOf(adapterColumnNames.get(i));
|
||||||
}
|
}
|
||||||
return dimLookup;
|
return dimLookup;
|
||||||
}
|
}
|
||||||
|
@ -1226,25 +1057,6 @@ public class IndexMerger
|
||||||
IndexIO.checkFileSize(indexFile);
|
IndexIO.checkFileSize(indexFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected IndexSeeker[] toIndexSeekers(
|
|
||||||
List<IndexableAdapter> adapters,
|
|
||||||
ArrayList<Map<String, IntBuffer>> 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<String> dimValueLookup = adapters.get(i).getDimValueLookup(dimension);
|
|
||||||
seekers[i] = new IndexSeekerWithoutConversion(dimValueLookup == null ? 0 : dimValueLookup.size());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return seekers;
|
|
||||||
}
|
|
||||||
|
|
||||||
static interface IndexSeeker
|
static interface IndexSeeker
|
||||||
{
|
{
|
||||||
int NOT_EXIST = -1;
|
int NOT_EXIST = -1;
|
||||||
|
@ -1253,22 +1065,6 @@ public class IndexMerger
|
||||||
int seek(int dictId);
|
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
|
* Get old dictId from new dictId, and only support access in order
|
||||||
*/
|
*/
|
||||||
|
@ -1324,69 +1120,28 @@ public class IndexMerger
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class ConvertingIndexedInts implements Iterable<Integer>
|
|
||||||
{
|
|
||||||
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<Integer> iterator()
|
|
||||||
{
|
|
||||||
return Iterators.transform(
|
|
||||||
baseIndex.iterator(),
|
|
||||||
new Function<Integer, Integer>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Integer apply(@Nullable Integer input)
|
|
||||||
{
|
|
||||||
return conversionBuffer.get(input);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class MMappedIndexRowIterable implements Iterable<Rowboat>
|
public static class MMappedIndexRowIterable implements Iterable<Rowboat>
|
||||||
{
|
{
|
||||||
private final Iterable<Rowboat> index;
|
private final Iterable<Rowboat> index;
|
||||||
private final List<String> convertedDims;
|
private final List<String> convertedDims;
|
||||||
private final Map<String, IntBuffer> converters;
|
|
||||||
private final int indexNumber;
|
private final int indexNumber;
|
||||||
private final ArrayList<Boolean> convertMissingDimsFlags;
|
private final List<ColumnCapabilitiesImpl> dimCapabilities;
|
||||||
private static final int[] EMPTY_STR_DIM = new int[]{0};
|
private final List<DimensionMerger> mergers;
|
||||||
|
|
||||||
|
|
||||||
MMappedIndexRowIterable(
|
MMappedIndexRowIterable(
|
||||||
Iterable<Rowboat> index,
|
Iterable<Rowboat> index,
|
||||||
List<String> convertedDims,
|
List<String> convertedDims,
|
||||||
Map<String, IntBuffer> converters,
|
|
||||||
int indexNumber,
|
int indexNumber,
|
||||||
ArrayList<Boolean> convertMissingDimsFlags
|
final List<ColumnCapabilitiesImpl> dimCapabilities,
|
||||||
|
final List<DimensionMerger> mergers
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.index = index;
|
this.index = index;
|
||||||
this.convertedDims = convertedDims;
|
this.convertedDims = convertedDims;
|
||||||
this.converters = converters;
|
|
||||||
this.indexNumber = indexNumber;
|
this.indexNumber = indexNumber;
|
||||||
this.convertMissingDimsFlags = convertMissingDimsFlags;
|
this.dimCapabilities = dimCapabilities;
|
||||||
|
this.mergers = mergers;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Iterable<Rowboat> getIndex()
|
public Iterable<Rowboat> getIndex()
|
||||||
|
@ -1397,18 +1152,6 @@ public class IndexMerger
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Rowboat> iterator()
|
public Iterator<Rowboat> iterator()
|
||||||
{
|
{
|
||||||
final IntBuffer[] converterArray = FluentIterable
|
|
||||||
.from(convertedDims)
|
|
||||||
.transform(
|
|
||||||
new Function<String, IntBuffer>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public IntBuffer apply(String input)
|
|
||||||
{
|
|
||||||
return converters.get(input);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
).toArray(IntBuffer.class);
|
|
||||||
return Iterators.transform(
|
return Iterators.transform(
|
||||||
index.iterator(),
|
index.iterator(),
|
||||||
new Function<Rowboat, Rowboat>()
|
new Function<Rowboat, Rowboat>()
|
||||||
|
@ -1416,36 +1159,21 @@ public class IndexMerger
|
||||||
@Override
|
@Override
|
||||||
public Rowboat apply(@Nullable Rowboat input)
|
public Rowboat apply(@Nullable Rowboat input)
|
||||||
{
|
{
|
||||||
int[][] dims = input.getDims();
|
Object[] dims = input.getDims();
|
||||||
int[][] newDims = new int[convertedDims.size()][];
|
Object[] newDims = new Object[convertedDims.size()];
|
||||||
for (int i = 0; i < convertedDims.size(); ++i) {
|
for (int i = 0; i < convertedDims.size(); ++i) {
|
||||||
IntBuffer converter = converterArray[i];
|
|
||||||
if (i >= dims.length) {
|
if (i >= dims.length) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
newDims[i] = mergers.get(i).convertSegmentRowValuesToMergedRowValues(dims[i], indexNumber);
|
||||||
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]);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
final Rowboat retVal = new Rowboat(
|
final Rowboat retVal = new Rowboat(
|
||||||
input.getTimestamp(),
|
input.getTimestamp(),
|
||||||
newDims,
|
newDims,
|
||||||
input.getMetrics(),
|
input.getMetrics(),
|
||||||
input.getRowNum()
|
input.getRowNum(),
|
||||||
|
input.getHandlers()
|
||||||
);
|
);
|
||||||
|
|
||||||
retVal.addRow(indexNumber, input.getRowNum());
|
retVal.addRow(indexNumber, input.getRowNum());
|
||||||
|
@ -1533,7 +1261,8 @@ public class IndexMerger
|
||||||
lhs.getTimestamp(),
|
lhs.getTimestamp(),
|
||||||
lhs.getDims(),
|
lhs.getDims(),
|
||||||
metrics,
|
metrics,
|
||||||
lhs.getRowNum()
|
lhs.getRowNum(),
|
||||||
|
lhs.getHandlers()
|
||||||
);
|
);
|
||||||
|
|
||||||
for (Rowboat rowboat : Arrays.asList(lhs, rhs)) {
|
for (Rowboat rowboat : Arrays.asList(lhs, rhs)) {
|
||||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.segment;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.collect.Iterables;
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
|
@ -42,7 +41,6 @@ import com.metamx.common.ISE;
|
||||||
import com.metamx.common.io.smoosh.FileSmoosher;
|
import com.metamx.common.io.smoosh.FileSmoosher;
|
||||||
import com.metamx.common.io.smoosh.SmooshedWriter;
|
import com.metamx.common.io.smoosh.SmooshedWriter;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import io.druid.collections.CombiningIterable;
|
|
||||||
import io.druid.common.utils.JodaUtils;
|
import io.druid.common.utils.JodaUtils;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
|
@ -50,26 +48,15 @@ import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
||||||
import io.druid.segment.column.ColumnDescriptor;
|
import io.druid.segment.column.ColumnDescriptor;
|
||||||
import io.druid.segment.column.ValueType;
|
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.CompressedObjectStrategy;
|
||||||
import io.druid.segment.data.CompressedVSizeIndexedV3Writer;
|
|
||||||
import io.druid.segment.data.CompressedVSizeIntsIndexedWriter;
|
|
||||||
import io.druid.segment.data.CompressionFactory;
|
import io.druid.segment.data.CompressionFactory;
|
||||||
import io.druid.segment.data.GenericIndexed;
|
import io.druid.segment.data.GenericIndexed;
|
||||||
import io.druid.segment.data.GenericIndexedWriter;
|
|
||||||
import io.druid.segment.data.IOPeon;
|
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.TmpFileIOPeon;
|
||||||
import io.druid.segment.data.VSizeIndexedIntsWriter;
|
|
||||||
import io.druid.segment.data.VSizeIndexedWriter;
|
|
||||||
import io.druid.segment.serde.ComplexColumnPartSerde;
|
import io.druid.segment.serde.ComplexColumnPartSerde;
|
||||||
import io.druid.segment.serde.ComplexColumnSerializer;
|
import io.druid.segment.serde.ComplexColumnSerializer;
|
||||||
import io.druid.segment.serde.ComplexMetricSerde;
|
import io.druid.segment.serde.ComplexMetricSerde;
|
||||||
import io.druid.segment.serde.ComplexMetrics;
|
import io.druid.segment.serde.ComplexMetrics;
|
||||||
import io.druid.segment.serde.DictionaryEncodedColumnPartSerde;
|
|
||||||
import io.druid.segment.serde.FloatGenericColumnPartSerde;
|
import io.druid.segment.serde.FloatGenericColumnPartSerde;
|
||||||
import io.druid.segment.serde.LongGenericColumnPartSerde;
|
import io.druid.segment.serde.LongGenericColumnPartSerde;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
|
@ -79,11 +66,9 @@ import org.joda.time.Interval;
|
||||||
import java.io.ByteArrayOutputStream;
|
import java.io.ByteArrayOutputStream;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileOutputStream;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.nio.IntBuffer;
|
import java.nio.IntBuffer;
|
||||||
import java.nio.MappedByteBuffer;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -184,18 +169,20 @@ public class IndexMergerV9 extends IndexMerger
|
||||||
final List<ColumnCapabilitiesImpl> dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
final List<ColumnCapabilitiesImpl> dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
||||||
mergeCapabilities(adapters, mergedDimensions, metricsValueTypes, metricTypeNames, dimCapabilities);
|
mergeCapabilities(adapters, mergedDimensions, metricsValueTypes, metricTypeNames, dimCapabilities);
|
||||||
|
|
||||||
|
final DimensionHandler[] handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities);
|
||||||
|
final List<DimensionMerger> 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 **************/
|
/************* Setup Dim Conversions **************/
|
||||||
progress.progress();
|
progress.progress();
|
||||||
startTime = System.currentTimeMillis();
|
startTime = System.currentTimeMillis();
|
||||||
final Map<String, Integer> dimCardinalities = Maps.newHashMap();
|
|
||||||
final ArrayList<GenericIndexedWriter<String>> dimValueWriters = setupDimValueWriters(ioPeon, mergedDimensions);
|
|
||||||
final ArrayList<Map<String, IntBuffer>> dimConversions = Lists.newArrayListWithCapacity(adapters.size());
|
final ArrayList<Map<String, IntBuffer>> dimConversions = Lists.newArrayListWithCapacity(adapters.size());
|
||||||
final ArrayList<Boolean> dimensionSkipFlag = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
final ArrayList<Boolean> dimensionSkipFlag = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
||||||
final ArrayList<Boolean> dimHasNullFlags = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
|
||||||
final ArrayList<Boolean> convertMissingDimsFlags = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
final ArrayList<Boolean> convertMissingDimsFlags = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
||||||
writeDimValueAndSetupDimConversion(
|
writeDimValueAndSetupDimConversion(
|
||||||
adapters, progress, mergedDimensions, dimCardinalities, dimValueWriters, dimensionSkipFlag, dimConversions,
|
adapters, progress, mergedDimensions, mergers
|
||||||
convertMissingDimsFlags, dimHasNullFlags
|
|
||||||
);
|
);
|
||||||
log.info("Completed dim conversions in %,d millis.", System.currentTimeMillis() - startTime);
|
log.info("Completed dim conversions in %,d millis.", System.currentTimeMillis() - startTime);
|
||||||
|
|
||||||
|
@ -205,53 +192,43 @@ public class IndexMergerV9 extends IndexMerger
|
||||||
adapters,
|
adapters,
|
||||||
mergedDimensions,
|
mergedDimensions,
|
||||||
mergedMetrics,
|
mergedMetrics,
|
||||||
dimConversions,
|
rowMergerFn,
|
||||||
convertMissingDimsFlags,
|
dimCapabilities,
|
||||||
rowMergerFn
|
handlers,
|
||||||
|
mergers
|
||||||
);
|
);
|
||||||
final LongColumnSerializer timeWriter = setupTimeWriter(ioPeon, indexSpec);
|
final LongColumnSerializer timeWriter = setupTimeWriter(ioPeon, indexSpec);
|
||||||
final ArrayList<IndexedIntsWriter> dimWriters = setupDimensionWriters(
|
|
||||||
ioPeon, mergedDimensions, dimCapabilities, dimCardinalities, indexSpec
|
|
||||||
);
|
|
||||||
final ArrayList<GenericColumnSerializer> metWriters = setupMetricsWriters(
|
final ArrayList<GenericColumnSerializer> metWriters = setupMetricsWriters(
|
||||||
ioPeon, mergedMetrics, metricsValueTypes, metricTypeNames, indexSpec
|
ioPeon, mergedMetrics, metricsValueTypes, metricTypeNames, indexSpec
|
||||||
);
|
);
|
||||||
final List<IntBuffer> rowNumConversions = Lists.newArrayListWithCapacity(adapters.size());
|
final List<IntBuffer> rowNumConversions = Lists.newArrayListWithCapacity(adapters.size());
|
||||||
final ArrayList<MutableBitmap> nullRowsList = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
|
||||||
for (int i = 0; i < mergedDimensions.size(); ++i) {
|
|
||||||
nullRowsList.add(indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap());
|
|
||||||
}
|
|
||||||
mergeIndexesAndWriteColumns(
|
mergeIndexesAndWriteColumns(
|
||||||
adapters, progress, theRows, timeWriter, dimWriters, metWriters,
|
adapters, progress, theRows, timeWriter, metWriters, rowNumConversions, mergers
|
||||||
dimensionSkipFlag, rowNumConversions, nullRowsList, dimHasNullFlags
|
|
||||||
);
|
);
|
||||||
|
|
||||||
/************ Create Inverted Indexes *************/
|
/************ Create Inverted Indexes and Finalize Build Columns *************/
|
||||||
progress.progress();
|
final String section = "build inverted index and columns";
|
||||||
final ArrayList<GenericIndexedWriter<ImmutableBitmap>> bitmapIndexWriters = setupBitmapIndexWriters(
|
progress.startSection(section);
|
||||||
ioPeon, mergedDimensions, indexSpec
|
|
||||||
);
|
|
||||||
final ArrayList<ByteBufferWriter<ImmutableRTree>> spatialIndexWriters = setupSpatialIndexWriters(
|
|
||||||
ioPeon, mergedDimensions, indexSpec, dimCapabilities
|
|
||||||
);
|
|
||||||
makeInvertedIndexes(
|
|
||||||
adapters, progress, mergedDimensions, indexSpec, v9TmpDir, rowNumConversions,
|
|
||||||
nullRowsList, dimValueWriters, bitmapIndexWriters, spatialIndexWriters, dimConversions
|
|
||||||
);
|
|
||||||
|
|
||||||
/************ Finalize Build Columns *************/
|
|
||||||
progress.progress();
|
|
||||||
makeTimeColumn(v9Smoosher, progress, timeWriter);
|
makeTimeColumn(v9Smoosher, progress, timeWriter);
|
||||||
makeMetricsColumns(v9Smoosher, progress, mergedMetrics, metricsValueTypes, metricTypeNames, metWriters);
|
makeMetricsColumns(v9Smoosher, progress, mergedMetrics, metricsValueTypes, metricTypeNames, metWriters);
|
||||||
makeDimensionColumns(
|
|
||||||
v9Smoosher, progress, indexSpec, mergedDimensions, dimensionSkipFlag, dimCapabilities,
|
for(int i = 0; i < mergedDimensions.size(); i++) {
|
||||||
dimValueWriters, dimWriters, bitmapIndexWriters, spatialIndexWriters
|
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 **************/
|
/************* Make index.drd & metadata.drd files **************/
|
||||||
progress.progress();
|
progress.progress();
|
||||||
makeIndexBinary(
|
makeIndexBinary(
|
||||||
v9Smoosher, adapters, outDir, mergedDimensions, dimensionSkipFlag, mergedMetrics, progress, indexSpec
|
v9Smoosher, adapters, outDir, mergedDimensions, mergedMetrics, progress, indexSpec, mergers
|
||||||
);
|
);
|
||||||
makeMetadataBinary(v9Smoosher, progress, segmentMetadata);
|
makeMetadataBinary(v9Smoosher, progress, segmentMetadata);
|
||||||
|
|
||||||
|
@ -286,10 +263,10 @@ public class IndexMergerV9 extends IndexMerger
|
||||||
final List<IndexableAdapter> adapters,
|
final List<IndexableAdapter> adapters,
|
||||||
final File outDir,
|
final File outDir,
|
||||||
final List<String> mergedDimensions,
|
final List<String> mergedDimensions,
|
||||||
final ArrayList<Boolean> dimensionSkipFlag,
|
|
||||||
final List<String> mergedMetrics,
|
final List<String> mergedMetrics,
|
||||||
final ProgressIndicator progress,
|
final ProgressIndicator progress,
|
||||||
final IndexSpec indexSpec
|
final IndexSpec indexSpec,
|
||||||
|
final List<DimensionMerger> mergers
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final String section = "make index.drd";
|
final String section = "make index.drd";
|
||||||
|
@ -300,7 +277,7 @@ public class IndexMergerV9 extends IndexMerger
|
||||||
final Set<String> finalColumns = Sets.newLinkedHashSet();
|
final Set<String> finalColumns = Sets.newLinkedHashSet();
|
||||||
finalColumns.addAll(mergedMetrics);
|
finalColumns.addAll(mergedMetrics);
|
||||||
for (int i = 0; i < mergedDimensions.size(); ++i) {
|
for (int i = 0; i < mergedDimensions.size(); ++i) {
|
||||||
if (dimensionSkipFlag.get(i)) {
|
if (mergers.get(i).canSkip()) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
finalColumns.add(mergedDimensions.get(i));
|
finalColumns.add(mergedDimensions.get(i));
|
||||||
|
@ -343,68 +320,6 @@ public class IndexMergerV9 extends IndexMerger
|
||||||
progress.stopSection(section);
|
progress.stopSection(section);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void makeDimensionColumns(
|
|
||||||
final FileSmoosher v9Smoosher,
|
|
||||||
final ProgressIndicator progress,
|
|
||||||
final IndexSpec indexSpec,
|
|
||||||
final List<String> mergedDimensions,
|
|
||||||
final ArrayList<Boolean> dimensionSkipFlag,
|
|
||||||
final List<ColumnCapabilitiesImpl> dimCapabilities,
|
|
||||||
final ArrayList<GenericIndexedWriter<String>> dimValueWriters,
|
|
||||||
final ArrayList<IndexedIntsWriter> dimWriters,
|
|
||||||
final ArrayList<GenericIndexedWriter<ImmutableBitmap>> bitmapIndexWriters,
|
|
||||||
final ArrayList<ByteBufferWriter<ImmutableRTree>> 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<ImmutableBitmap> bitmapIndexWriter = bitmapIndexWriters.get(i);
|
|
||||||
final ByteBufferWriter<ImmutableRTree> 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(
|
private void makeMetricsColumns(
|
||||||
final FileSmoosher v9Smoosher,
|
final FileSmoosher v9Smoosher,
|
||||||
final ProgressIndicator progress,
|
final ProgressIndicator progress,
|
||||||
|
@ -464,7 +379,6 @@ public class IndexMergerV9 extends IndexMerger
|
||||||
progress.stopSection(section);
|
progress.stopSection(section);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private void makeTimeColumn(
|
private void makeTimeColumn(
|
||||||
final FileSmoosher v9Smoosher,
|
final FileSmoosher v9Smoosher,
|
||||||
final ProgressIndicator progress,
|
final ProgressIndicator progress,
|
||||||
|
@ -514,173 +428,14 @@ public class IndexMergerV9 extends IndexMerger
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void makeInvertedIndexes(
|
|
||||||
final List<IndexableAdapter> adapters,
|
|
||||||
final ProgressIndicator progress,
|
|
||||||
final List<String> mergedDimensions,
|
|
||||||
final IndexSpec indexSpec,
|
|
||||||
final File v9OutDir,
|
|
||||||
final List<IntBuffer> rowNumConversions,
|
|
||||||
final ArrayList<MutableBitmap> nullRowsList,
|
|
||||||
final ArrayList<GenericIndexedWriter<String>> dimValueWriters,
|
|
||||||
final ArrayList<GenericIndexedWriter<ImmutableBitmap>> bitmapIndexWriters,
|
|
||||||
final ArrayList<ByteBufferWriter<ImmutableRTree>> spatialIndexWriters,
|
|
||||||
final ArrayList<Map<String, IntBuffer>> 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<String> dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY);
|
|
||||||
|
|
||||||
ByteBufferWriter<ImmutableRTree> 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<Iterable<Integer>> 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.<Integer>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<String> 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<GenericIndexedWriter<ImmutableBitmap>> setupBitmapIndexWriters(
|
|
||||||
final IOPeon ioPeon,
|
|
||||||
final List<String> mergedDimensions,
|
|
||||||
final IndexSpec indexSpec
|
|
||||||
) throws IOException
|
|
||||||
{
|
|
||||||
ArrayList<GenericIndexedWriter<ImmutableBitmap>> writers = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
|
||||||
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
|
|
||||||
for (String dimension : mergedDimensions) {
|
|
||||||
GenericIndexedWriter<ImmutableBitmap> writer = new GenericIndexedWriter<>(
|
|
||||||
ioPeon, String.format("%s.inverted", dimension), bitmapSerdeFactory.getObjectStrategy()
|
|
||||||
);
|
|
||||||
writer.open();
|
|
||||||
writers.add(writer);
|
|
||||||
}
|
|
||||||
return writers;
|
|
||||||
}
|
|
||||||
|
|
||||||
private ArrayList<ByteBufferWriter<ImmutableRTree>> setupSpatialIndexWriters(
|
|
||||||
final IOPeon ioPeon,
|
|
||||||
final List<String> mergedDimensions,
|
|
||||||
final IndexSpec indexSpec,
|
|
||||||
final List<ColumnCapabilitiesImpl> dimCapabilities
|
|
||||||
) throws IOException
|
|
||||||
{
|
|
||||||
ArrayList<ByteBufferWriter<ImmutableRTree>> 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<ImmutableRTree> 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(
|
private void mergeIndexesAndWriteColumns(
|
||||||
final List<IndexableAdapter> adapters,
|
final List<IndexableAdapter> adapters,
|
||||||
final ProgressIndicator progress,
|
final ProgressIndicator progress,
|
||||||
final Iterable<Rowboat> theRows,
|
final Iterable<Rowboat> theRows,
|
||||||
final LongColumnSerializer timeWriter,
|
final LongColumnSerializer timeWriter,
|
||||||
final ArrayList<IndexedIntsWriter> dimWriters,
|
|
||||||
final ArrayList<GenericColumnSerializer> metWriters,
|
final ArrayList<GenericColumnSerializer> metWriters,
|
||||||
final ArrayList<Boolean> dimensionSkipFlag,
|
|
||||||
final List<IntBuffer> rowNumConversions,
|
final List<IntBuffer> rowNumConversions,
|
||||||
final ArrayList<MutableBitmap> nullRowsList,
|
final List<DimensionMerger> mergers
|
||||||
final ArrayList<Boolean> dimHasNullFlags
|
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final String section = "walk through and merge rows";
|
final String section = "walk through and merge rows";
|
||||||
|
@ -704,19 +459,13 @@ public class IndexMergerV9 extends IndexMerger
|
||||||
metWriters.get(i).serialize(metrics[i]);
|
metWriters.get(i).serialize(metrics[i]);
|
||||||
}
|
}
|
||||||
|
|
||||||
int[][] dims = theRow.getDims();
|
Object[] dims = theRow.getDims();
|
||||||
for (int i = 0; i < dims.length; ++i) {
|
for (int i = 0; i < dims.length; ++i) {
|
||||||
if (dimensionSkipFlag.get(i)) {
|
DimensionMerger merger = mergers.get(i);
|
||||||
|
if (merger.canSkip()) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
if (dims[i] == null || dims[i].length == 0) {
|
merger.processMergedRow(dims[i]);
|
||||||
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]);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for (Map.Entry<Integer, TreeSet<Integer>> comprisedRow : theRow.getComprisedRows().entrySet()) {
|
for (Map.Entry<Integer, TreeSet<Integer>> comprisedRow : theRow.getComprisedRows().entrySet()) {
|
||||||
|
@ -791,153 +540,18 @@ public class IndexMergerV9 extends IndexMerger
|
||||||
return metWriters;
|
return metWriters;
|
||||||
}
|
}
|
||||||
|
|
||||||
private ArrayList<IndexedIntsWriter> setupDimensionWriters(
|
|
||||||
final IOPeon ioPeon,
|
|
||||||
final List<String> mergedDimensions,
|
|
||||||
final List<ColumnCapabilitiesImpl> dimCapabilities,
|
|
||||||
final Map<String, Integer> dimCardinalities,
|
|
||||||
final IndexSpec indexSpec
|
|
||||||
) throws IOException
|
|
||||||
{
|
|
||||||
ArrayList<IndexedIntsWriter> 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<GenericIndexedWriter<String>> setupDimValueWriters(
|
|
||||||
final IOPeon ioPeon,
|
|
||||||
final List<String> mergedDimensions
|
|
||||||
)
|
|
||||||
throws IOException
|
|
||||||
{
|
|
||||||
ArrayList<GenericIndexedWriter<String>> dimValueWriters = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
|
||||||
for (String dimension : mergedDimensions) {
|
|
||||||
final GenericIndexedWriter<String> writer = new GenericIndexedWriter<>(
|
|
||||||
ioPeon, String.format("%s.dim_values", dimension), GenericIndexed.STRING_STRATEGY
|
|
||||||
);
|
|
||||||
writer.open();
|
|
||||||
dimValueWriters.add(writer);
|
|
||||||
}
|
|
||||||
return dimValueWriters;
|
|
||||||
}
|
|
||||||
|
|
||||||
private void writeDimValueAndSetupDimConversion(
|
private void writeDimValueAndSetupDimConversion(
|
||||||
final List<IndexableAdapter> indexes,
|
final List<IndexableAdapter> indexes,
|
||||||
final ProgressIndicator progress,
|
final ProgressIndicator progress,
|
||||||
final List<String> mergedDimensions,
|
final List<String> mergedDimensions,
|
||||||
final Map<String, Integer> dimensionCardinalities,
|
final List<DimensionMerger> mergers
|
||||||
final ArrayList<GenericIndexedWriter<String>> dimValueWriters,
|
|
||||||
final ArrayList<Boolean> dimensionSkipFlag,
|
|
||||||
final List<Map<String, IntBuffer>> dimConversions,
|
|
||||||
final ArrayList<Boolean> convertMissingDimsFlags,
|
|
||||||
final ArrayList<Boolean> dimHasNullFlags
|
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final String section = "setup dimension conversions";
|
final String section = "setup dimension conversions";
|
||||||
progress.startSection(section);
|
progress.startSection(section);
|
||||||
|
|
||||||
for (int i = 0; i < indexes.size(); ++i) {
|
|
||||||
dimConversions.add(Maps.<String, IntBuffer>newHashMap());
|
|
||||||
}
|
|
||||||
|
|
||||||
for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) {
|
for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) {
|
||||||
long dimStartTime = System.currentTimeMillis();
|
mergers.get(dimIndex).writeMergedValueMetadata(indexes);
|
||||||
String dimension = mergedDimensions.get(dimIndex);
|
|
||||||
boolean dimHasNull = false;
|
|
||||||
boolean dimHasValues = false;
|
|
||||||
boolean dimAbsentFromSomeIndex = false;
|
|
||||||
|
|
||||||
int numMergeIndex = 0;
|
|
||||||
Indexed<String> dimValueLookup = null;
|
|
||||||
Indexed<String>[] dimValueLookups = new Indexed[indexes.size() + 1];
|
|
||||||
for (int i = 0; i < indexes.size(); i++) {
|
|
||||||
Indexed<String> 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<String> 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);
|
|
||||||
}
|
}
|
||||||
progress.stopSection(section);
|
progress.stopSection(section);
|
||||||
}
|
}
|
||||||
|
@ -956,7 +570,7 @@ public class IndexMergerV9 extends IndexMerger
|
||||||
ColumnCapabilitiesImpl mergedCapabilities = capabilitiesMap.get(dimension);
|
ColumnCapabilitiesImpl mergedCapabilities = capabilitiesMap.get(dimension);
|
||||||
if (mergedCapabilities == null) {
|
if (mergedCapabilities == null) {
|
||||||
mergedCapabilities = new ColumnCapabilitiesImpl();
|
mergedCapabilities = new ColumnCapabilitiesImpl();
|
||||||
mergedCapabilities.setType(ValueType.STRING);
|
mergedCapabilities.setType(null);
|
||||||
}
|
}
|
||||||
capabilitiesMap.put(dimension, mergedCapabilities.merge(adapter.getCapabilities(dimension)));
|
capabilitiesMap.put(dimension, mergedCapabilities.merge(adapter.getCapabilities(dimension)));
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,8 @@ import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An adapter to an index
|
* An adapter to an index
|
||||||
*/
|
*/
|
||||||
|
@ -37,7 +39,7 @@ public interface IndexableAdapter
|
||||||
|
|
||||||
Indexed<String> getMetricNames();
|
Indexed<String> getMetricNames();
|
||||||
|
|
||||||
Indexed<String> getDimValueLookup(String dimension);
|
Indexed<Comparable> getDimValueLookup(String dimension);
|
||||||
|
|
||||||
Iterable<Rowboat> getRows();
|
Iterable<Rowboat> getRows();
|
||||||
|
|
||||||
|
@ -48,4 +50,6 @@ public interface IndexableAdapter
|
||||||
ColumnCapabilities getCapabilities(String column);
|
ColumnCapabilities getCapabilities(String column);
|
||||||
|
|
||||||
Metadata getMetadata();
|
Metadata getMetadata();
|
||||||
|
|
||||||
|
Map<String, DimensionHandler> getDimensionHandlers();
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -36,6 +37,7 @@ public interface QueryableIndex extends ColumnSelector, Closeable
|
||||||
public Indexed<String> getAvailableDimensions();
|
public Indexed<String> getAvailableDimensions();
|
||||||
public BitmapFactory getBitmapFactoryForDimensions();
|
public BitmapFactory getBitmapFactoryForDimensions();
|
||||||
public Metadata getMetadata();
|
public Metadata getMetadata();
|
||||||
|
public Map<String, DimensionHandler> getDimensionHandlers();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The close method shouldn't actually be here as this is nasty. We will adjust it in the future.
|
* The close method shouldn't actually be here as this is nasty. We will adjust it in the future.
|
||||||
|
|
|
@ -36,7 +36,6 @@ import io.druid.segment.column.GenericColumn;
|
||||||
import io.druid.segment.column.IndexedFloatsGenericColumn;
|
import io.druid.segment.column.IndexedFloatsGenericColumn;
|
||||||
import io.druid.segment.column.IndexedLongsGenericColumn;
|
import io.druid.segment.column.IndexedLongsGenericColumn;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
import io.druid.segment.data.ArrayBasedIndexedInts;
|
|
||||||
import io.druid.segment.data.BitmapCompressedIndexedInts;
|
import io.druid.segment.data.BitmapCompressedIndexedInts;
|
||||||
import io.druid.segment.data.EmptyIndexedInts;
|
import io.druid.segment.data.EmptyIndexedInts;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
|
@ -46,9 +45,11 @@ import io.druid.segment.data.ListIndexed;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.NoSuchElementException;
|
import java.util.NoSuchElementException;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
@ -76,10 +77,11 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||||
|
|
||||||
if (col == null) {
|
if (col == null) {
|
||||||
log.warn("Wtf!? column[%s] didn't exist!?!?!?", dim);
|
log.warn("Wtf!? column[%s] didn't exist!?!?!?", dim);
|
||||||
} else if (col.getDictionaryEncoding() != null) {
|
|
||||||
availableDimensions.add(dim);
|
|
||||||
} else {
|
} 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 class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Indexed<String> getDimValueLookup(String dimension)
|
public Indexed<Comparable> getDimValueLookup(String dimension)
|
||||||
{
|
{
|
||||||
final Column column = input.getColumn(dimension);
|
final Column column = input.getColumn(dimension);
|
||||||
|
|
||||||
|
@ -131,12 +133,12 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
return new Indexed<String>()
|
return new Indexed<Comparable>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Class<? extends String> getClazz()
|
public Class<? extends Comparable> getClazz()
|
||||||
{
|
{
|
||||||
return String.class;
|
return Comparable.class;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -146,19 +148,19 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String get(int index)
|
public Comparable get(int index)
|
||||||
{
|
{
|
||||||
return dict.lookupName(index);
|
return dict.lookupName(index);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int indexOf(String value)
|
public int indexOf(Comparable value)
|
||||||
{
|
{
|
||||||
return dict.lookupId(value);
|
return dict.lookupId(value);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<String> iterator()
|
public Iterator<Comparable> iterator()
|
||||||
{
|
{
|
||||||
return IndexedIterable.create(this).iterator();
|
return IndexedIterable.create(this).iterator();
|
||||||
}
|
}
|
||||||
|
@ -177,28 +179,31 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||||
{
|
{
|
||||||
final GenericColumn timestamps = input.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn();
|
final GenericColumn timestamps = input.getColumn(Column.TIME_COLUMN_NAME).getGenericColumn();
|
||||||
final Object[] metrics;
|
final Object[] metrics;
|
||||||
|
final Closeable[] columns;
|
||||||
final DictionaryEncodedColumn[] dictionaryEncodedColumns;
|
|
||||||
|
|
||||||
final int numMetrics = getMetricNames().size();
|
final int numMetrics = getMetricNames().size();
|
||||||
|
|
||||||
|
final DimensionHandler[] handlers = new DimensionHandler[availableDimensions.size()];
|
||||||
|
Collection<DimensionHandler> handlerSet = input.getDimensionHandlers().values();
|
||||||
|
|
||||||
int currRow = 0;
|
int currRow = 0;
|
||||||
boolean done = false;
|
boolean done = false;
|
||||||
|
|
||||||
{
|
{
|
||||||
this.dictionaryEncodedColumns = FluentIterable
|
handlerSet.toArray(handlers);
|
||||||
.from(getDimensionNames())
|
this.columns = FluentIterable
|
||||||
|
.from(handlerSet)
|
||||||
.transform(
|
.transform(
|
||||||
new Function<String, DictionaryEncodedColumn>()
|
new Function<DimensionHandler, Closeable>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public DictionaryEncodedColumn apply(String dimName)
|
public Closeable apply(DimensionHandler handler)
|
||||||
{
|
{
|
||||||
return input.getColumn(dimName)
|
Column column = input.getColumn(handler.getDimensionName());
|
||||||
.getDictionaryEncoding();
|
return handler.getSubColumn(column);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
).toArray(DictionaryEncodedColumn.class);
|
).toArray(Closeable.class);
|
||||||
|
|
||||||
final Indexed<String> availableMetrics = getMetricNames();
|
final Indexed<String> availableMetrics = getMetricNames();
|
||||||
metrics = new Object[availableMetrics.size()];
|
metrics = new Object[availableMetrics.size()];
|
||||||
|
@ -230,10 +235,8 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||||
CloseQuietly.close((Closeable) metric);
|
CloseQuietly.close((Closeable) metric);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
for (Object dimension : dictionaryEncodedColumns) {
|
for (Closeable dimension : columns) {
|
||||||
if (dimension instanceof Closeable) {
|
CloseQuietly.close(dimension);
|
||||||
CloseQuietly.close((Closeable) dimension);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
done = true;
|
done = true;
|
||||||
}
|
}
|
||||||
|
@ -247,22 +250,11 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||||
throw new NoSuchElementException();
|
throw new NoSuchElementException();
|
||||||
}
|
}
|
||||||
|
|
||||||
final int[][] dims = new int[dictionaryEncodedColumns.length][];
|
final Object[] dims = new Object[columns.length];
|
||||||
int dimIndex = 0;
|
int dimIndex = 0;
|
||||||
for (final DictionaryEncodedColumn dict : dictionaryEncodedColumns) {
|
for (final Closeable column : columns) {
|
||||||
final IndexedInts dimVals;
|
dims[dimIndex] = handlers[dimIndex].getRowValueArrayFromColumn(column, currRow);
|
||||||
if (dict.hasMultipleValues()) {
|
dimIndex++;
|
||||||
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;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
Object[] metricArray = new Object[numMetrics];
|
Object[] metricArray = new Object[numMetrics];
|
||||||
|
@ -277,7 +269,7 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
final Rowboat retVal = new Rowboat(
|
final Rowboat retVal = new Rowboat(
|
||||||
timestamps.getLongSingleValueRow(currRow), dims, metricArray, currRow
|
timestamps.getLongSingleValueRow(currRow), dims, metricArray, currRow, handlers
|
||||||
);
|
);
|
||||||
|
|
||||||
++currRow;
|
++currRow;
|
||||||
|
@ -361,4 +353,10 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||||
{
|
{
|
||||||
return metadata;
|
return metadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, DimensionHandler> getDimensionHandlers()
|
||||||
|
{
|
||||||
|
return input.getDimensionHandlers();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -192,6 +192,12 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
return getColumnCapabilites(index, column);
|
return getColumnCapabilites(index, column);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, DimensionHandler> getDimensionHandlers()
|
||||||
|
{
|
||||||
|
return index.getDimensionHandlers();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getColumnTypeName(String columnName)
|
public String getColumnTypeName(String columnName)
|
||||||
{
|
{
|
||||||
|
@ -462,13 +468,13 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimension);
|
DictionaryEncodedColumn<String> cachedColumn = dictionaryColumnCache.get(dimension);
|
||||||
if (cachedColumn == null) {
|
if (cachedColumn == null) {
|
||||||
cachedColumn = columnDesc.getDictionaryEncoding();
|
cachedColumn = columnDesc.getDictionaryEncoding();
|
||||||
dictionaryColumnCache.put(dimension, cachedColumn);
|
dictionaryColumnCache.put(dimension, cachedColumn);
|
||||||
}
|
}
|
||||||
|
|
||||||
final DictionaryEncodedColumn column = cachedColumn;
|
final DictionaryEncodedColumn<String> column = cachedColumn;
|
||||||
|
|
||||||
if (column == null) {
|
if (column == null) {
|
||||||
return NULL_DIMENSION_SELECTOR;
|
return NULL_DIMENSION_SELECTOR;
|
||||||
|
@ -739,7 +745,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
if (cachedColumnVals instanceof DictionaryEncodedColumn) {
|
if (cachedColumnVals instanceof DictionaryEncodedColumn) {
|
||||||
final DictionaryEncodedColumn columnVals = (DictionaryEncodedColumn) cachedColumnVals;
|
final DictionaryEncodedColumn<String> columnVals = (DictionaryEncodedColumn) cachedColumnVals;
|
||||||
if (columnVals.hasMultipleValues()) {
|
if (columnVals.hasMultipleValues()) {
|
||||||
return new ObjectColumnSelector<Object>()
|
return new ObjectColumnSelector<Object>()
|
||||||
{
|
{
|
||||||
|
|
|
@ -32,22 +32,25 @@ import java.util.TreeSet;
|
||||||
public class Rowboat implements Comparable<Rowboat>
|
public class Rowboat implements Comparable<Rowboat>
|
||||||
{
|
{
|
||||||
private final long timestamp;
|
private final long timestamp;
|
||||||
private final int[][] dims;
|
private final Object[] dims;
|
||||||
private final Object[] metrics;
|
private final Object[] metrics;
|
||||||
private final int rowNum;
|
private final int rowNum;
|
||||||
private final Map<Integer, TreeSet<Integer>> comprisedRows;
|
private final Map<Integer, TreeSet<Integer>> comprisedRows;
|
||||||
|
private final DimensionHandler[] handlers;
|
||||||
|
|
||||||
public Rowboat(
|
public Rowboat(
|
||||||
long timestamp,
|
long timestamp,
|
||||||
int[][] dims,
|
Object[] dims,
|
||||||
Object[] metrics,
|
Object[] metrics,
|
||||||
int rowNum
|
int rowNum,
|
||||||
|
DimensionHandler[] handlers
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.timestamp = timestamp;
|
this.timestamp = timestamp;
|
||||||
this.dims = dims;
|
this.dims = dims;
|
||||||
this.metrics = metrics;
|
this.metrics = metrics;
|
||||||
this.rowNum = rowNum;
|
this.rowNum = rowNum;
|
||||||
|
this.handlers = handlers;
|
||||||
|
|
||||||
this.comprisedRows = Maps.newHashMap();
|
this.comprisedRows = Maps.newHashMap();
|
||||||
}
|
}
|
||||||
|
@ -57,7 +60,7 @@ public class Rowboat implements Comparable<Rowboat>
|
||||||
return timestamp;
|
return timestamp;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int[][] getDims()
|
public Object[] getDims()
|
||||||
{
|
{
|
||||||
return dims;
|
return dims;
|
||||||
}
|
}
|
||||||
|
@ -82,6 +85,11 @@ public class Rowboat implements Comparable<Rowboat>
|
||||||
return comprisedRows;
|
return comprisedRows;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public DimensionHandler[] getHandlers()
|
||||||
|
{
|
||||||
|
return handlers;
|
||||||
|
}
|
||||||
|
|
||||||
public int getRowNum()
|
public int getRowNum()
|
||||||
{
|
{
|
||||||
return rowNum;
|
return rowNum;
|
||||||
|
@ -98,8 +106,8 @@ public class Rowboat implements Comparable<Rowboat>
|
||||||
|
|
||||||
int index = 0;
|
int index = 0;
|
||||||
while (retVal == 0 && index < dims.length) {
|
while (retVal == 0 && index < dims.length) {
|
||||||
int[] lhsVals = dims[index];
|
Object lhsVals = dims[index];
|
||||||
int[] rhsVals = rhs.dims[index];
|
Object rhsVals = rhs.dims[index];
|
||||||
|
|
||||||
if (lhsVals == null) {
|
if (lhsVals == null) {
|
||||||
if (rhsVals == null) {
|
if (rhsVals == null) {
|
||||||
|
@ -113,13 +121,8 @@ public class Rowboat implements Comparable<Rowboat>
|
||||||
return 1;
|
return 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
retVal = Ints.compare(lhsVals.length, rhsVals.length);
|
DimensionHandler handler = handlers[index];
|
||||||
|
retVal = handler.compareSortedEncodedArrays(lhsVals, rhsVals);
|
||||||
int valsIndex = 0;
|
|
||||||
while (retVal == 0 && valsIndex < lhsVals.length) {
|
|
||||||
retVal = Ints.compare(lhsVals[valsIndex], rhsVals[valsIndex]);
|
|
||||||
++valsIndex;
|
|
||||||
}
|
|
||||||
++index;
|
++index;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -26,6 +26,8 @@ import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class RowboatFilteringIndexAdapter implements IndexableAdapter
|
public class RowboatFilteringIndexAdapter implements IndexableAdapter
|
||||||
|
@ -64,7 +66,7 @@ public class RowboatFilteringIndexAdapter implements IndexableAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Indexed<String> getDimValueLookup(String dimension)
|
public Indexed<Comparable> getDimValueLookup(String dimension)
|
||||||
{
|
{
|
||||||
return baseAdapter.getDimValueLookup(dimension);
|
return baseAdapter.getDimValueLookup(dimension);
|
||||||
}
|
}
|
||||||
|
@ -98,4 +100,10 @@ public class RowboatFilteringIndexAdapter implements IndexableAdapter
|
||||||
{
|
{
|
||||||
return baseAdapter.getMetadata();
|
return baseAdapter.getMetadata();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, DimensionHandler> getDimensionHandlers()
|
||||||
|
{
|
||||||
|
return baseAdapter.getDimensionHandlers();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,9 +20,11 @@
|
||||||
package io.druid.segment;
|
package io.druid.segment;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
import com.metamx.collections.bitmap.BitmapFactory;
|
import com.metamx.collections.bitmap.BitmapFactory;
|
||||||
import com.metamx.common.io.smoosh.SmooshedFileMapper;
|
import com.metamx.common.io.smoosh.SmooshedFileMapper;
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
@ -40,6 +42,7 @@ public class SimpleQueryableIndex implements QueryableIndex
|
||||||
private final Map<String, Column> columns;
|
private final Map<String, Column> columns;
|
||||||
private final SmooshedFileMapper fileMapper;
|
private final SmooshedFileMapper fileMapper;
|
||||||
private final Metadata metadata;
|
private final Metadata metadata;
|
||||||
|
private final Map<String, DimensionHandler> dimensionHandlers;
|
||||||
|
|
||||||
public SimpleQueryableIndex(
|
public SimpleQueryableIndex(
|
||||||
Interval dataInterval,
|
Interval dataInterval,
|
||||||
|
@ -59,6 +62,8 @@ public class SimpleQueryableIndex implements QueryableIndex
|
||||||
this.columns = columns;
|
this.columns = columns;
|
||||||
this.fileMapper = fileMapper;
|
this.fileMapper = fileMapper;
|
||||||
this.metadata = metadata;
|
this.metadata = metadata;
|
||||||
|
this.dimensionHandlers = Maps.newLinkedHashMap();
|
||||||
|
initDimensionHandlers();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -108,4 +113,19 @@ public class SimpleQueryableIndex implements QueryableIndex
|
||||||
{
|
{
|
||||||
return metadata;
|
return metadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, DimensionHandler> 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);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,8 @@ import io.druid.segment.data.Indexed;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public interface StorageAdapter extends CursorFactory
|
public interface StorageAdapter extends CursorFactory
|
||||||
|
@ -48,6 +50,7 @@ public interface StorageAdapter extends CursorFactory
|
||||||
public Comparable getMaxValue(String column);
|
public Comparable getMaxValue(String column);
|
||||||
public Capabilities getCapabilities();
|
public Capabilities getCapabilities();
|
||||||
public ColumnCapabilities getColumnCapabilities(String column);
|
public ColumnCapabilities getColumnCapabilities(String column);
|
||||||
|
public Map<String, DimensionHandler> getDimensionHandlers();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Like {@link ColumnCapabilities#getType()}, but may return a more descriptive string for complex columns.
|
* Like {@link ColumnCapabilities#getType()}, but may return a more descriptive string for complex columns.
|
||||||
|
|
|
@ -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<Integer, int[], String>
|
||||||
|
{
|
||||||
|
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<String> lhsEncodings,
|
||||||
|
Indexed<String> 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<Integer, int[], String> 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<Object, String> STRING_TRANSFORMER = new Function<Object, String>()
|
||||||
|
{
|
||||||
|
@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<Integer> ENCODED_COMPARATOR = new Comparator<Integer>()
|
||||||
|
{
|
||||||
|
@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<String> UNENCODED_COMPARATOR = new Comparator<String>()
|
||||||
|
{
|
||||||
|
@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);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
|
@ -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<Integer, int[], String>
|
||||||
|
{
|
||||||
|
private static final Logger log = new Logger(StringDimensionIndexer.class);
|
||||||
|
|
||||||
|
private static class DimensionDictionary
|
||||||
|
{
|
||||||
|
private String minValue = null;
|
||||||
|
private String maxValue = null;
|
||||||
|
|
||||||
|
private final Map<String, Integer> valueToId = Maps.newHashMap();
|
||||||
|
|
||||||
|
private final List<String> 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<String> sortedVals;
|
||||||
|
private final int[] idToIndex;
|
||||||
|
private final int[] indexToId;
|
||||||
|
|
||||||
|
public SortedDimensionDictionary(List<String> idToValue, int length)
|
||||||
|
{
|
||||||
|
Map<String, Integer> 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<Object> 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<String> getSortedIndexedValues()
|
||||||
|
{
|
||||||
|
updateSortedLookup();
|
||||||
|
return new Indexed<String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class<? extends String> 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<String> 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<Integer> 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<Integer> 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<Integer> 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<Comparable> 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<String> 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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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<int[]>
|
||||||
|
{
|
||||||
|
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<Integer> listToWrite = (vals == null)
|
||||||
|
? null
|
||||||
|
: Ints.asList(vals);
|
||||||
|
encodedValueWriterV8.add(listToWrite);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeIndexes(List<IntBuffer> 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<String> 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<ImmutableRTree>(
|
||||||
|
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<Iterable<Integer>> 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.<Integer>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<String> 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<FileOutputStream> 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();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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<int[]>
|
||||||
|
{
|
||||||
|
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<String> dictionaryWriter;
|
||||||
|
protected GenericIndexedWriter<ImmutableBitmap> bitmapWriter;
|
||||||
|
protected ByteBufferWriter<ImmutableRTree> spatialWriter;
|
||||||
|
protected ArrayList<IntBuffer> 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<IndexableAdapter> 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<IndexableAdapter> 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<String> dimValueLookup = null;
|
||||||
|
Indexed<String>[] dimValueLookups = new Indexed[adapters.size() + 1];
|
||||||
|
for (int i = 0; i < adapters.size(); i++) {
|
||||||
|
Indexed<String> 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<IntBuffer> 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<String> 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<Iterable<Integer>> 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.<Integer>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<String> 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<Integer>
|
||||||
|
{
|
||||||
|
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<Integer> iterator()
|
||||||
|
{
|
||||||
|
return Iterators.transform(
|
||||||
|
baseIndex.iterator(),
|
||||||
|
new Function<Integer, Integer>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Integer apply(@Nullable Integer input)
|
||||||
|
{
|
||||||
|
return conversionBuffer.get(input);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected IndexSeeker[] toIndexSeekers(
|
||||||
|
List<IndexableAdapter> adapters,
|
||||||
|
ArrayList<IntBuffer> 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<String> dimValueLookup = (Indexed) adapters.get(i).getDimValueLookup(dimension);
|
||||||
|
seekers[i] = new IndexSeekerWithoutConversion(dimValueLookup == null ? 0 : dimValueLookup.size());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return seekers;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected boolean isNullColumn(Iterable<String> dimValues)
|
||||||
|
{
|
||||||
|
if (dimValues == null) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
for (String val : dimValues) {
|
||||||
|
if (val != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
|
@ -25,13 +25,13 @@ import java.io.Closeable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public interface DictionaryEncodedColumn extends Closeable
|
public interface DictionaryEncodedColumn<ActualType extends Comparable> extends Closeable
|
||||||
{
|
{
|
||||||
public int length();
|
public int length();
|
||||||
public boolean hasMultipleValues();
|
public boolean hasMultipleValues();
|
||||||
public int getSingleValueRow(int rowNum);
|
public int getSingleValueRow(int rowNum);
|
||||||
public IndexedInts getMultiValueRow(int rowNum);
|
public IndexedInts getMultiValueRow(int rowNum);
|
||||||
public String lookupName(int id);
|
public ActualType lookupName(int id);
|
||||||
public int lookupId(String name);
|
public int lookupId(ActualType name);
|
||||||
public int getCardinality();
|
public int getCardinality();
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,7 +30,7 @@ import java.io.IOException;
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class SimpleDictionaryEncodedColumn
|
public class SimpleDictionaryEncodedColumn
|
||||||
implements DictionaryEncodedColumn
|
implements DictionaryEncodedColumn<String>
|
||||||
{
|
{
|
||||||
private final IndexedInts column;
|
private final IndexedInts column;
|
||||||
private final IndexedMultivalue<IndexedInts> multiValueColumn;
|
private final IndexedMultivalue<IndexedInts> multiValueColumn;
|
||||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.segment.incremental;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Strings;
|
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
@ -29,12 +28,10 @@ import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Iterators;
|
import com.google.common.collect.Iterators;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Ordering;
|
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.parsers.ParseException;
|
|
||||||
import io.druid.data.input.InputRow;
|
import io.druid.data.input.InputRow;
|
||||||
import io.druid.data.input.MapBasedRow;
|
import io.druid.data.input.MapBasedRow;
|
||||||
import io.druid.data.input.Row;
|
import io.druid.data.input.Row;
|
||||||
|
@ -47,6 +44,9 @@ import io.druid.query.aggregation.PostAggregator;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
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.DimensionSelector;
|
||||||
import io.druid.segment.FloatColumnSelector;
|
import io.druid.segment.FloatColumnSelector;
|
||||||
import io.druid.segment.LongColumnSelector;
|
import io.druid.segment.LongColumnSelector;
|
||||||
|
@ -67,9 +67,9 @@ import javax.annotation.Nullable;
|
||||||
import javax.annotation.concurrent.GuardedBy;
|
import javax.annotation.concurrent.GuardedBy;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.lang.reflect.Array;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.Deque;
|
import java.util.Deque;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -101,73 +101,11 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
.put(DimensionSchema.ValueType.STRING, ValueType.STRING)
|
.put(DimensionSchema.ValueType.STRING, ValueType.STRING)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
private static final Function<Object, String> STRING_TRANSFORMER = new Function<Object, String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String apply(final Object o)
|
|
||||||
{
|
|
||||||
return o == null ? null : String.valueOf(o);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
private static final Function<Object, Long> LONG_TRANSFORMER = new Function<Object, Long>()
|
|
||||||
{
|
|
||||||
@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<Object, Float> FLOAT_TRANSFORMER = new Function<Object, Float>()
|
|
||||||
{
|
|
||||||
@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<ValueType, Function> VALUE_TRANSFORMS = ImmutableMap.<ValueType, Function>builder()
|
|
||||||
.put(ValueType.LONG, LONG_TRANSFORMER)
|
|
||||||
.put(ValueType.FLOAT, FLOAT_TRANSFORMER)
|
|
||||||
.put(ValueType.STRING, STRING_TRANSFORMER)
|
|
||||||
.build();
|
|
||||||
|
|
||||||
public static ColumnSelectorFactory makeColumnSelectorFactory(
|
public static ColumnSelectorFactory makeColumnSelectorFactory(
|
||||||
final AggregatorFactory agg,
|
final AggregatorFactory agg,
|
||||||
final Supplier<InputRow> in,
|
final Supplier<InputRow> in,
|
||||||
final boolean deserializeComplexMetrics
|
final boolean deserializeComplexMetrics,
|
||||||
|
final Map<String, ColumnCapabilitiesImpl> columnCapabilities
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new ColumnSelectorFactory()
|
return new ColumnSelectorFactory()
|
||||||
|
@ -369,12 +307,8 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
private final Map<String, MetricDesc> metricDescs;
|
private final Map<String, MetricDesc> metricDescs;
|
||||||
|
|
||||||
private final Map<String, DimensionDesc> dimensionDescs;
|
private final Map<String, DimensionDesc> dimensionDescs;
|
||||||
|
private final List<DimensionDesc> dimensionDescsList;
|
||||||
private final Map<String, ColumnCapabilitiesImpl> columnCapabilities;
|
private final Map<String, ColumnCapabilitiesImpl> columnCapabilities;
|
||||||
private final List<DimDim> dimValues;
|
|
||||||
|
|
||||||
// looks need a configuration
|
|
||||||
private final Ordering<Comparable> ordering = Ordering.natural().nullsFirst();
|
|
||||||
|
|
||||||
private final AtomicInteger numEntries = new AtomicInteger();
|
private final AtomicInteger numEntries = new AtomicInteger();
|
||||||
|
|
||||||
// This is modified on add() in a critical section.
|
// This is modified on add() in a critical section.
|
||||||
|
@ -412,6 +346,7 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
this.deserializeComplexMetrics = deserializeComplexMetrics;
|
this.deserializeComplexMetrics = deserializeComplexMetrics;
|
||||||
this.reportParseExceptions = reportParseExceptions;
|
this.reportParseExceptions = reportParseExceptions;
|
||||||
|
|
||||||
|
this.columnCapabilities = Maps.newHashMap();
|
||||||
this.metadata = new Metadata()
|
this.metadata = new Metadata()
|
||||||
.setAggregators(getCombiningAggregators(metrics))
|
.setAggregators(getCombiningAggregators(metrics))
|
||||||
.setTimestampSpec(incrementalIndexSchema.getTimestampSpec())
|
.setTimestampSpec(incrementalIndexSchema.getTimestampSpec())
|
||||||
|
@ -419,7 +354,6 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
.setRollup(this.rollup);
|
.setRollup(this.rollup);
|
||||||
|
|
||||||
this.aggs = initAggs(metrics, rowSupplier, deserializeComplexMetrics);
|
this.aggs = initAggs(metrics, rowSupplier, deserializeComplexMetrics);
|
||||||
this.columnCapabilities = Maps.newHashMap();
|
|
||||||
|
|
||||||
this.metricDescs = Maps.newLinkedHashMap();
|
this.metricDescs = Maps.newLinkedHashMap();
|
||||||
for (AggregatorFactory metric : metrics) {
|
for (AggregatorFactory metric : metrics) {
|
||||||
|
@ -429,20 +363,20 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
}
|
}
|
||||||
|
|
||||||
DimensionsSpec dimensionsSpec = incrementalIndexSchema.getDimensionsSpec();
|
DimensionsSpec dimensionsSpec = incrementalIndexSchema.getDimensionsSpec();
|
||||||
|
|
||||||
this.dimensionDescs = Maps.newLinkedHashMap();
|
this.dimensionDescs = Maps.newLinkedHashMap();
|
||||||
this.dimValues = Collections.synchronizedList(Lists.<DimDim>newArrayList());
|
|
||||||
|
|
||||||
|
this.dimensionDescsList = new ArrayList<>();
|
||||||
for (DimensionSchema dimSchema : dimensionsSpec.getDimensions()) {
|
for (DimensionSchema dimSchema : dimensionsSpec.getDimensions()) {
|
||||||
ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl();
|
|
||||||
ValueType type = TYPE_MAP.get(dimSchema.getValueType());
|
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)) {
|
if (dimSchema.getTypeName().equals(DimensionSchema.SPATIAL_TYPE_NAME)) {
|
||||||
capabilities.setHasSpatialIndexes(true);
|
capabilities.setHasSpatialIndexes(true);
|
||||||
} else {
|
} 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
|
//__time capabilites
|
||||||
|
@ -457,28 +391,6 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
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()
|
public boolean isRollup()
|
||||||
{
|
{
|
||||||
return rollup;
|
return rollup;
|
||||||
|
@ -521,7 +433,6 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
@Override
|
@Override
|
||||||
public void close()
|
public void close()
|
||||||
{
|
{
|
||||||
dimValues.clear();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public InputRow formatRow(InputRow row)
|
public InputRow formatRow(InputRow row)
|
||||||
|
@ -553,31 +464,16 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
return TYPE_MAP.get(singleVal.getClass());
|
return TYPE_MAP.get(singleVal.getClass());
|
||||||
}
|
}
|
||||||
|
|
||||||
private List<Comparable> getRowDimensionAsComparables(InputRow row, String dimension, ValueType type)
|
|
||||||
{
|
|
||||||
final Object dimVal = row.getRaw(dimension);
|
|
||||||
final Function transformer = VALUE_TRANSFORMS.get(type);
|
|
||||||
final List<Comparable> 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;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Map<String, DimensionDesc> getDimensionDescs()
|
public Map<String, DimensionDesc> getDimensionDescs()
|
||||||
{
|
{
|
||||||
return dimensionDescs;
|
return dimensionDescs;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Map<String, ColumnCapabilitiesImpl> getColumnCapabilities()
|
||||||
|
{
|
||||||
|
return columnCapabilities;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adds a new row. The row might correspond with another row that already exists, in which case this will
|
* Adds a new row. The row might correspond with another row that already exists, in which case this will
|
||||||
* update that row instead of inserting a new one.
|
* update that row instead of inserting a new one.
|
||||||
|
@ -617,45 +513,45 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
|
|
||||||
final List<String> rowDimensions = row.getDimensions();
|
final List<String> rowDimensions = row.getDimensions();
|
||||||
|
|
||||||
int[][] dims;
|
Object[] dims;
|
||||||
List<int[]> overflow = null;
|
List<Object> overflow = null;
|
||||||
List<ValueType> overflowTypes = null;
|
|
||||||
synchronized (dimensionDescs) {
|
synchronized (dimensionDescs) {
|
||||||
dims = new int[dimensionDescs.size()][];
|
dims = new Object[dimensionDescs.size()];
|
||||||
for (String dimension : rowDimensions) {
|
for (String dimension : rowDimensions) {
|
||||||
List<Comparable> dimensionValues;
|
boolean wasNewDim = false;
|
||||||
|
|
||||||
ColumnCapabilitiesImpl capabilities;
|
ColumnCapabilitiesImpl capabilities;
|
||||||
final ValueType valType;
|
ValueType valType = null;
|
||||||
DimensionDesc desc = dimensionDescs.get(dimension);
|
DimensionDesc desc = dimensionDescs.get(dimension);
|
||||||
if (desc != null) {
|
if (desc != null) {
|
||||||
capabilities = desc.getCapabilities();
|
capabilities = desc.getCapabilities();
|
||||||
} else {
|
} else {
|
||||||
|
wasNewDim = true;
|
||||||
capabilities = columnCapabilities.get(dimension);
|
capabilities = columnCapabilities.get(dimension);
|
||||||
if (capabilities == null) {
|
if (capabilities == null) {
|
||||||
capabilities = new ColumnCapabilitiesImpl();
|
capabilities = new ColumnCapabilitiesImpl();
|
||||||
// For schemaless type discovery, assume everything is a String for now, can change later.
|
// For schemaless type discovery, assume everything is a String for now, can change later.
|
||||||
capabilities.setType(ValueType.STRING);
|
capabilities.setType(ValueType.STRING);
|
||||||
|
capabilities.setDictionaryEncoded(true);
|
||||||
|
capabilities.setHasBitmapIndexes(true);
|
||||||
columnCapabilities.put(dimension, capabilities);
|
columnCapabilities.put(dimension, capabilities);
|
||||||
}
|
}
|
||||||
|
DimensionHandler handler = DimensionHandlerUtil.getHandlerFromCapabilities(dimension, capabilities);
|
||||||
|
desc = addNewDimension(dimension, capabilities, handler);
|
||||||
}
|
}
|
||||||
valType = capabilities.getType();
|
DimensionHandler handler = desc.getHandler();
|
||||||
dimensionValues = getRowDimensionAsComparables(row, dimension, valType);
|
DimensionIndexer indexer = desc.getIndexer();
|
||||||
|
Object dimsKey = indexer.processRowValsToUnsortedEncodedArray(row.getRaw(dimension));
|
||||||
|
|
||||||
// Set column capabilities as data is coming in
|
// 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);
|
capabilities.setHasMultipleValues(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (desc == null) {
|
if (wasNewDim) {
|
||||||
desc = addNewDimension(dimension, capabilities);
|
|
||||||
|
|
||||||
if (overflow == null) {
|
if (overflow == null) {
|
||||||
overflow = Lists.newArrayList();
|
overflow = Lists.newArrayList();
|
||||||
overflowTypes = Lists.newArrayList();
|
|
||||||
}
|
}
|
||||||
overflow.add(getDimVals(desc.getValues(), dimensionValues));
|
overflow.add(dimsKey);
|
||||||
overflowTypes.add(valType);
|
|
||||||
} else if (desc.getIndex() > dims.length || dims[desc.getIndex()] != null) {
|
} 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,
|
* index > dims.length requires that we saw this dimension and added it to the dimensionOrder map,
|
||||||
|
@ -668,14 +564,14 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
*/
|
*/
|
||||||
throw new ISE("Dimension[%s] occurred more than once in InputRow", dimension);
|
throw new ISE("Dimension[%s] occurred more than once in InputRow", dimension);
|
||||||
} else {
|
} else {
|
||||||
dims[desc.getIndex()] = getDimVals(desc.getValues(), dimensionValues);
|
dims[desc.getIndex()] = dimsKey;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (overflow != null) {
|
if (overflow != null) {
|
||||||
// Merge overflow and non-overflow
|
// 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);
|
System.arraycopy(dims, 0, newDims, 0, dims.length);
|
||||||
for (int i = 0; i < overflow.size(); ++i) {
|
for (int i = 0; i < overflow.size(); ++i) {
|
||||||
newDims[dims.length + i] = overflow.get(i);
|
newDims[dims.length + i] = overflow.get(i);
|
||||||
|
@ -684,7 +580,7 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
}
|
}
|
||||||
|
|
||||||
long truncated = gran.truncate(row.getTimestampFromEpoch());
|
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)
|
private synchronized void updateMaxIngestedTime(DateTime eventTime)
|
||||||
|
@ -714,33 +610,6 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
return getFacts().getMaxTimeMillis();
|
return getFacts().getMaxTimeMillis();
|
||||||
}
|
}
|
||||||
|
|
||||||
private int[] getDimVals(final DimDim dimLookup, final List<Comparable> 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()
|
public AggregatorType[] getAggs()
|
||||||
{
|
{
|
||||||
return aggs;
|
return aggs;
|
||||||
|
@ -809,10 +678,10 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
return isEmpty() ? null : new DateTime(getMaxTimeMillis());
|
return isEmpty() ? null : new DateTime(getMaxTimeMillis());
|
||||||
}
|
}
|
||||||
|
|
||||||
public DimDim getDimensionValues(String dimension)
|
public Integer getDimensionIndex(String dimension)
|
||||||
{
|
{
|
||||||
DimensionDesc dimSpec = getDimension(dimension);
|
DimensionDesc dimSpec = getDimension(dimension);
|
||||||
return dimSpec == null ? null : dimSpec.getValues();
|
return dimSpec == null ? null : dimSpec.getIndex();
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<String> getDimensionOrder()
|
public List<String> getDimensionOrder()
|
||||||
|
@ -822,12 +691,21 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
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
|
* Currently called to initialize IncrementalIndex dimension order during index creation
|
||||||
* Index dimension ordering could be changed to initialize from DimensionsSpec after resolution of
|
* Index dimension ordering could be changed to initialize from DimensionsSpec after resolution of
|
||||||
* https://github.com/druid-io/druid/issues/2011
|
* https://github.com/druid-io/druid/issues/2011
|
||||||
*/
|
*/
|
||||||
public void loadDimensionIterable(Iterable<String> oldDimensionOrder)
|
public void loadDimensionIterable(Iterable<String> oldDimensionOrder, Map<String, ColumnCapabilitiesImpl> oldColumnCapabilities)
|
||||||
{
|
{
|
||||||
synchronized (dimensionDescs) {
|
synchronized (dimensionDescs) {
|
||||||
if (!dimensionDescs.isEmpty()) {
|
if (!dimensionDescs.isEmpty()) {
|
||||||
|
@ -835,25 +713,21 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
}
|
}
|
||||||
for (String dim : oldDimensionOrder) {
|
for (String dim : oldDimensionOrder) {
|
||||||
if (dimensionDescs.get(dim) == null) {
|
if (dimensionDescs.get(dim) == null) {
|
||||||
ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl();
|
ColumnCapabilitiesImpl capabilities = oldColumnCapabilities.get(dim);
|
||||||
capabilities.setType(ValueType.STRING);
|
|
||||||
columnCapabilities.put(dim, capabilities);
|
columnCapabilities.put(dim, capabilities);
|
||||||
addNewDimension(dim, capabilities);
|
DimensionHandler handler = DimensionHandlerUtil.getHandlerFromCapabilities(dim, capabilities);
|
||||||
|
addNewDimension(dim, capabilities, handler);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@GuardedBy("dimensionDescs")
|
@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);
|
DimensionDesc desc = new DimensionDesc(dimensionDescs.size(), dim, capabilities, handler);
|
||||||
if (dimValues.size() != desc.getIndex()) {
|
|
||||||
throw new ISE("dimensionDescs and dimValues for [%s] is out of sync!!", dim);
|
|
||||||
}
|
|
||||||
|
|
||||||
dimensionDescs.put(dim, desc);
|
dimensionDescs.put(dim, desc);
|
||||||
dimValues.add(desc.getValues());
|
dimensionDescsList.add(desc);
|
||||||
return desc;
|
return desc;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -892,6 +766,15 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
return combiningAggregators;
|
return combiningAggregators;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Map<String, DimensionHandler> getDimensionHandlers()
|
||||||
|
{
|
||||||
|
Map<String, DimensionHandler> handlers = Maps.newLinkedHashMap();
|
||||||
|
for (DimensionDesc desc : dimensionDescsList) {
|
||||||
|
handlers.put(desc.getName(), desc.getHandler());
|
||||||
|
}
|
||||||
|
return handlers;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Row> iterator()
|
public Iterator<Row> iterator()
|
||||||
{
|
{
|
||||||
|
@ -917,38 +800,24 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
final TimeAndDims timeAndDims = input.getKey();
|
final TimeAndDims timeAndDims = input.getKey();
|
||||||
final int rowOffset = input.getValue();
|
final int rowOffset = input.getValue();
|
||||||
|
|
||||||
int[][] theDims = timeAndDims.getDims(); //TODO: remove dictionary encoding for numerics later
|
Object[] theDims = timeAndDims.getDims();
|
||||||
|
|
||||||
Map<String, Object> theVals = Maps.newLinkedHashMap();
|
Map<String, Object> theVals = Maps.newLinkedHashMap();
|
||||||
for (int i = 0; i < theDims.length; ++i) {
|
for (int i = 0; i < theDims.length; ++i) {
|
||||||
int[] dim = theDims[i];
|
Object dim = theDims[i];
|
||||||
DimensionDesc dimensionDesc = dimensions.get(i);
|
DimensionDesc dimensionDesc = dimensions.get(i);
|
||||||
if (dimensionDesc == null) {
|
if (dimensionDesc == null) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
ValueType type = dimensionDesc.getCapabilities().getType();
|
|
||||||
String dimensionName = dimensionDesc.getName();
|
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);
|
theVals.put(dimensionName, null);
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
if (dim.length == 1) {
|
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||||
Comparable val = dimensionDesc.getValues().getValue(dim[0]);
|
Object rowVals = indexer.convertUnsortedEncodedArrayToActualArrayOrList(dim, true);
|
||||||
if (type == ValueType.STRING) {
|
theVals.put(dimensionName, rowVals);
|
||||||
val = Strings.nullToEmpty((String) val);
|
|
||||||
}
|
|
||||||
theVals.put(dimensionName, val);
|
|
||||||
} else {
|
|
||||||
List<Comparable> dimVals = new ArrayList<Comparable>(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);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
AggregatorType[] aggs = getAggsForRow(rowOffset);
|
AggregatorType[] aggs = getAggsForRow(rowOffset);
|
||||||
|
@ -979,15 +848,17 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
{
|
{
|
||||||
private final int index;
|
private final int index;
|
||||||
private final String name;
|
private final String name;
|
||||||
private final DimDim values;
|
|
||||||
private final ColumnCapabilitiesImpl capabilities;
|
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.index = index;
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.values = values;
|
|
||||||
this.capabilities = capabilities;
|
this.capabilities = capabilities;
|
||||||
|
this.handler = handler;
|
||||||
|
this.indexer = handler.makeIndexer();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getIndex()
|
public int getIndex()
|
||||||
|
@ -1000,15 +871,20 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
return name;
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
public DimDim getValues()
|
|
||||||
{
|
|
||||||
return values;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ColumnCapabilitiesImpl getCapabilities()
|
public ColumnCapabilitiesImpl getCapabilities()
|
||||||
{
|
{
|
||||||
return capabilities;
|
return capabilities;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public DimensionHandler getHandler()
|
||||||
|
{
|
||||||
|
return handler;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DimensionIndexer getIndexer()
|
||||||
|
{
|
||||||
|
return indexer;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final class MetricDesc
|
public static final class MetricDesc
|
||||||
|
@ -1058,151 +934,29 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static interface DimDim<T extends Comparable<? super T>>
|
public static final class TimeAndDims
|
||||||
{
|
|
||||||
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<T extends Comparable<? super T>>
|
|
||||||
{
|
|
||||||
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<String>
|
|
||||||
{
|
|
||||||
private final DimDim<String> 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<String>
|
|
||||||
{
|
|
||||||
private final SortedDimLookup<String> 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
|
|
||||||
{
|
{
|
||||||
private final long timestamp;
|
private final long timestamp;
|
||||||
private final int[][] dims;
|
private final Object[] dims;
|
||||||
|
private final List<DimensionDesc> dimensionDescsList;
|
||||||
|
|
||||||
TimeAndDims(
|
TimeAndDims(
|
||||||
long timestamp,
|
long timestamp,
|
||||||
int[][] dims
|
Object[] dims,
|
||||||
|
List<DimensionDesc> dimensionDescsList
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.timestamp = timestamp;
|
this.timestamp = timestamp;
|
||||||
this.dims = dims;
|
this.dims = dims;
|
||||||
|
this.dimensionDescsList = dimensionDescsList;
|
||||||
}
|
}
|
||||||
|
|
||||||
long getTimestamp()
|
public long getTimestamp()
|
||||||
{
|
{
|
||||||
return timestamp;
|
return timestamp;
|
||||||
}
|
}
|
||||||
|
|
||||||
int[][] getDims()
|
public Object[] getDims()
|
||||||
{
|
{
|
||||||
return dims;
|
return dims;
|
||||||
}
|
}
|
||||||
|
@ -1213,12 +967,12 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
return "TimeAndDims{" +
|
return "TimeAndDims{" +
|
||||||
"timestamp=" + new DateTime(timestamp) +
|
"timestamp=" + new DateTime(timestamp) +
|
||||||
", dims=" + Lists.transform(
|
", dims=" + Lists.transform(
|
||||||
Arrays.asList(dims), new Function<int[], Object>()
|
Arrays.asList(dims), new Function<Object, Object>()
|
||||||
{
|
{
|
||||||
@Override
|
@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("null");
|
||||||
}
|
}
|
||||||
return Arrays.asList(input);
|
return Arrays.asList(input);
|
||||||
|
@ -1246,7 +1000,8 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
for (int i = 0; i < dims.length; i++) {
|
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;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1258,7 +1013,8 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
{
|
{
|
||||||
int hash = (int) timestamp;
|
int hash = (int) timestamp;
|
||||||
for (int i = 0; i < dims.length; i++) {
|
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;
|
return hash;
|
||||||
}
|
}
|
||||||
|
@ -1266,17 +1022,17 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
|
|
||||||
protected final Comparator<TimeAndDims> dimsComparator()
|
protected final Comparator<TimeAndDims> dimsComparator()
|
||||||
{
|
{
|
||||||
return new TimeAndDimsComp(dimValues);
|
return new TimeAndDimsComp(dimensionDescsList);
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
static final class TimeAndDimsComp implements Comparator<TimeAndDims>
|
static final class TimeAndDimsComp implements Comparator<TimeAndDims>
|
||||||
{
|
{
|
||||||
private final List<DimDim> dimValues;
|
private List<DimensionDesc> dimensionDescs;
|
||||||
|
|
||||||
public TimeAndDimsComp(List<DimDim> dimValues)
|
public TimeAndDimsComp(List<DimensionDesc> dimDescs)
|
||||||
{
|
{
|
||||||
this.dimValues = dimValues;
|
this.dimensionDescs = dimDescs;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1287,8 +1043,8 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
|
|
||||||
int index = 0;
|
int index = 0;
|
||||||
while (retVal == 0 && index < numComparisons) {
|
while (retVal == 0 && index < numComparisons) {
|
||||||
final int[] lhsIdxs = lhs.dims[index];
|
final Object lhsIdxs = lhs.dims[index];
|
||||||
final int[] rhsIdxs = rhs.dims[index];
|
final Object rhsIdxs = rhs.dims[index];
|
||||||
|
|
||||||
if (lhsIdxs == null) {
|
if (lhsIdxs == null) {
|
||||||
if (rhsIdxs == null) {
|
if (rhsIdxs == null) {
|
||||||
|
@ -1302,22 +1058,8 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
return 1;
|
return 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
retVal = Ints.compare(lhsIdxs.length, rhsIdxs.length);
|
final DimensionIndexer indexer = dimensionDescs.get(index).getIndexer();
|
||||||
|
retVal = indexer.compareUnsortedEncodedArrays(lhsIdxs, rhsIdxs);
|
||||||
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;
|
|
||||||
}
|
|
||||||
++index;
|
++index;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1416,8 +1158,9 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
{
|
{
|
||||||
private final boolean sortFacts;
|
private final boolean sortFacts;
|
||||||
private final ConcurrentMap<TimeAndDims, Integer> facts;
|
private final ConcurrentMap<TimeAndDims, Integer> facts;
|
||||||
|
private final List<DimensionDesc> dimensionDescsList;
|
||||||
|
|
||||||
public RollupFactsHolder(boolean sortFacts, Comparator<TimeAndDims> timeAndDimsComparator)
|
public RollupFactsHolder(boolean sortFacts, Comparator<TimeAndDims> timeAndDimsComparator, List<DimensionDesc> dimensionDescsList)
|
||||||
{
|
{
|
||||||
this.sortFacts = sortFacts;
|
this.sortFacts = sortFacts;
|
||||||
if (sortFacts) {
|
if (sortFacts) {
|
||||||
|
@ -1425,6 +1168,7 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
} else {
|
} else {
|
||||||
this.facts = new ConcurrentHashMap<>();
|
this.facts = new ConcurrentHashMap<>();
|
||||||
}
|
}
|
||||||
|
this.dimensionDescsList = dimensionDescsList;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1473,8 +1217,8 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
if (!sortFacts) {
|
if (!sortFacts) {
|
||||||
throw new UnsupportedOperationException("can't get timeRange from unsorted facts data.");
|
throw new UnsupportedOperationException("can't get timeRange from unsorted facts data.");
|
||||||
}
|
}
|
||||||
TimeAndDims start = new TimeAndDims(timeStart, new int[][]{});
|
TimeAndDims start = new TimeAndDims(timeStart, new Object[]{}, dimensionDescsList);
|
||||||
TimeAndDims end = new TimeAndDims(timeEnd, new int[][]{});
|
TimeAndDims end = new TimeAndDims(timeEnd, new Object[]{}, dimensionDescsList);
|
||||||
ConcurrentNavigableMap<TimeAndDims, Integer> subMap =
|
ConcurrentNavigableMap<TimeAndDims, Integer> subMap =
|
||||||
((ConcurrentNavigableMap<TimeAndDims, Integer>) facts).subMap(start, end);
|
((ConcurrentNavigableMap<TimeAndDims, Integer>) facts).subMap(start, end);
|
||||||
final Map<TimeAndDims, Integer> rangeMap = descending ? subMap.descendingMap() : subMap;
|
final Map<TimeAndDims, Integer> rangeMap = descending ? subMap.descendingMap() : subMap;
|
||||||
|
|
|
@ -22,10 +22,11 @@ package io.druid.segment.incremental;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.collect.Iterators;
|
import com.google.common.collect.Iterators;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Sets;
|
|
||||||
import com.metamx.collections.bitmap.BitmapFactory;
|
import com.metamx.collections.bitmap.BitmapFactory;
|
||||||
import com.metamx.collections.bitmap.MutableBitmap;
|
import com.metamx.collections.bitmap.MutableBitmap;
|
||||||
import com.metamx.common.logger.Logger;
|
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.IndexableAdapter;
|
||||||
import io.druid.segment.Metadata;
|
import io.druid.segment.Metadata;
|
||||||
import io.druid.segment.Rowboat;
|
import io.druid.segment.Rowboat;
|
||||||
|
@ -33,7 +34,6 @@ import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.data.EmptyIndexedInts;
|
import io.druid.segment.data.EmptyIndexedInts;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.data.IndexedIterable;
|
|
||||||
import io.druid.segment.data.ListIndexed;
|
import io.druid.segment.data.ListIndexed;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.roaringbitmap.IntIterator;
|
import org.roaringbitmap.IntIterator;
|
||||||
|
@ -42,7 +42,6 @@ import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
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 static final Logger log = new Logger(IncrementalIndexAdapter.class);
|
||||||
private final Interval dataInterval;
|
private final Interval dataInterval;
|
||||||
private final IncrementalIndex<?> index;
|
private final IncrementalIndex<?> index;
|
||||||
private final Set<String> hasNullValueDimensions;
|
private final Map<String, DimensionAccessor> accessors;
|
||||||
|
|
||||||
private final Map<String, DimensionIndexer> indexers;
|
private class DimensionAccessor
|
||||||
|
|
||||||
private class DimensionIndexer
|
|
||||||
{
|
{
|
||||||
private final IncrementalIndex.DimensionDesc dimensionDesc;
|
private final IncrementalIndex.DimensionDesc dimensionDesc;
|
||||||
private final MutableBitmap[] invertedIndexes;
|
private final MutableBitmap[] invertedIndexes;
|
||||||
|
private final DimensionIndexer indexer;
|
||||||
|
|
||||||
private IncrementalIndex.SortedDimLookup dimLookup;
|
public DimensionAccessor(IncrementalIndex.DimensionDesc dimensionDesc)
|
||||||
|
|
||||||
public DimensionIndexer(IncrementalIndex.DimensionDesc dimensionDesc)
|
|
||||||
{
|
{
|
||||||
this.dimensionDesc = dimensionDesc;
|
this.dimensionDesc = dimensionDesc;
|
||||||
this.invertedIndexes = new MutableBitmap[dimensionDesc.getValues().size() + 1];
|
this.indexer = dimensionDesc.getIndexer();
|
||||||
}
|
if(dimensionDesc.getCapabilities().hasBitmapIndexes()) {
|
||||||
|
this.invertedIndexes = new MutableBitmap[indexer.getCardinality() + 1];
|
||||||
private IncrementalIndex.DimDim getDimValues()
|
} else {
|
||||||
{
|
this.invertedIndexes = null;
|
||||||
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();
|
|
||||||
}
|
}
|
||||||
return dimLookup;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -102,42 +86,34 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
* in row 2. So we should iterate all rows to determine whether one dimension contains
|
* in row 2. So we should iterate all rows to determine whether one dimension contains
|
||||||
* a null value.
|
* a null value.
|
||||||
*/
|
*/
|
||||||
this.hasNullValueDimensions = Sets.newHashSet();
|
|
||||||
|
|
||||||
final List<IncrementalIndex.DimensionDesc> dimensions = index.getDimensions();
|
final List<IncrementalIndex.DimensionDesc> dimensions = index.getDimensions();
|
||||||
|
|
||||||
indexers = Maps.newHashMapWithExpectedSize(dimensions.size());
|
accessors = Maps.newHashMapWithExpectedSize(dimensions.size());
|
||||||
for (IncrementalIndex.DimensionDesc dimension : dimensions) {
|
for (IncrementalIndex.DimensionDesc dimension : dimensions) {
|
||||||
indexers.put(dimension.getName(), new DimensionIndexer(dimension));
|
accessors.put(dimension.getName(), new DimensionAccessor(dimension));
|
||||||
}
|
}
|
||||||
|
|
||||||
int rowNum = 0;
|
int rowNum = 0;
|
||||||
for (IncrementalIndex.TimeAndDims timeAndDims : index.getFacts().keySet()) {
|
for (IncrementalIndex.TimeAndDims timeAndDims : index.getFacts().keySet()) {
|
||||||
final int[][] dims = timeAndDims.getDims();
|
final Object[] dims = timeAndDims.getDims();
|
||||||
|
|
||||||
for (IncrementalIndex.DimensionDesc dimension : dimensions) {
|
for (IncrementalIndex.DimensionDesc dimension : dimensions) {
|
||||||
final int dimIndex = dimension.getIndex();
|
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) {
|
if (dimIndex >= dims.length || dims[dimIndex] == null) {
|
||||||
hasNullValueDimensions.add(dimension.getName());
|
accessor.indexer.processRowValsToUnsortedEncodedArray(null);
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
final IncrementalIndex.DimDim values = dimension.getValues();
|
final ColumnCapabilities capabilities = dimension.getCapabilities();
|
||||||
if (hasNullValue(values, dims[dimIndex])) {
|
|
||||||
hasNullValueDimensions.add(dimension.getName());
|
|
||||||
}
|
|
||||||
|
|
||||||
final MutableBitmap[] bitmapIndexes = indexer.invertedIndexes;
|
if(capabilities.hasBitmapIndexes()) {
|
||||||
|
final MutableBitmap[] bitmapIndexes = accessor.invertedIndexes;
|
||||||
for (Comparable dimIdxComparable : dims[dimIndex]) {
|
final DimensionIndexer indexer = accessor.indexer;
|
||||||
Integer dimIdx = (Integer) dimIdxComparable;
|
indexer.fillBitmapsFromUnsortedEncodedArray(dims[dimIndex], rowNum, bitmapIndexes, bitmapFactory);
|
||||||
if (bitmapIndexes[dimIdx] == null) {
|
|
||||||
bitmapIndexes[dimIdx] = bitmapFactory.makeEmptyMutableBitmap();
|
|
||||||
}
|
|
||||||
bitmapIndexes[dimIdx].add(rowNum);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
++rowNum;
|
++rowNum;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -167,50 +143,16 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Indexed<String> getDimValueLookup(String dimension)
|
public Indexed<Comparable> getDimValueLookup(String dimension)
|
||||||
{
|
{
|
||||||
final DimensionIndexer indexer = indexers.get(dimension);
|
final DimensionAccessor accessor = accessors.get(dimension);
|
||||||
if (indexer == null) {
|
if (accessor == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
final IncrementalIndex.DimDim dimDim = indexer.getDimValues();
|
|
||||||
final IncrementalIndex.SortedDimLookup dimLookup = indexer.getDimLookup();
|
|
||||||
|
|
||||||
return new Indexed<String>()
|
final DimensionIndexer indexer = accessor.dimensionDesc.getIndexer();
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Class<? extends String> getClazz()
|
|
||||||
{
|
|
||||||
return String.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
return indexer.getSortedIndexedValues();
|
||||||
public int size()
|
|
||||||
{
|
|
||||||
return dimLookup.size();
|
|
||||||
}
|
|
||||||
|
|
||||||
@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<String> iterator()
|
|
||||||
{
|
|
||||||
return IndexedIterable.create(this).iterator();
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -222,9 +164,11 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
public Iterator<Rowboat> iterator()
|
public Iterator<Rowboat> iterator()
|
||||||
{
|
{
|
||||||
final List<IncrementalIndex.DimensionDesc> dimensions = index.getDimensions();
|
final List<IncrementalIndex.DimensionDesc> 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) {
|
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 class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
public Rowboat apply(Map.Entry<IncrementalIndex.TimeAndDims, Integer> input)
|
public Rowboat apply(Map.Entry<IncrementalIndex.TimeAndDims, Integer> input)
|
||||||
{
|
{
|
||||||
final IncrementalIndex.TimeAndDims timeAndDims = input.getKey();
|
final IncrementalIndex.TimeAndDims timeAndDims = input.getKey();
|
||||||
final int[][] dimValues = timeAndDims.getDims();
|
final Object[] dimValues = timeAndDims.getDims();
|
||||||
final int rowOffset = input.getValue();
|
final int rowOffset = input.getValue();
|
||||||
|
|
||||||
int[][] dims = new int[dimValues.length][];
|
Object[] dims = new Object[dimValues.length];
|
||||||
for (IncrementalIndex.DimensionDesc dimension : dimensions) {
|
for (IncrementalIndex.DimensionDesc dimension : dimensions) {
|
||||||
final int dimIndex = dimension.getIndex();
|
final int dimIndex = dimension.getIndex();
|
||||||
|
|
||||||
|
@ -253,17 +197,9 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
dims[dimIndex] = new int[dimValues[dimIndex].length];
|
final DimensionIndexer indexer = indexers[dimIndex];
|
||||||
|
Object sortedDimVals = indexer.convertUnsortedEncodedArrayToSortedEncodedArray(dimValues[dimIndex]);
|
||||||
if (dimIndex >= dims.length || dims[dimIndex] == null) {
|
dims[dimIndex] = sortedDimVals;
|
||||||
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.
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
Object[] metrics = new Object[index.getMetricAggs().length];
|
Object[] metrics = new Object[index.getMetricAggs().length];
|
||||||
|
@ -275,7 +211,8 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
timeAndDims.getTimestamp(),
|
timeAndDims.getTimestamp(),
|
||||||
dims,
|
dims,
|
||||||
metrics,
|
metrics,
|
||||||
count++
|
count++,
|
||||||
|
handlers
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -287,14 +224,19 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
@Override
|
@Override
|
||||||
public IndexedInts getBitmapIndex(String dimension, int index)
|
public IndexedInts getBitmapIndex(String dimension, int index)
|
||||||
{
|
{
|
||||||
DimensionIndexer accessor = indexers.get(dimension);
|
DimensionAccessor accessor = accessors.get(dimension);
|
||||||
if (accessor == null) {
|
if (accessor == null) {
|
||||||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||||
}
|
}
|
||||||
|
ColumnCapabilities capabilities = accessor.dimensionDesc.getCapabilities();
|
||||||
|
DimensionIndexer indexer = accessor.dimensionDesc.getIndexer();
|
||||||
|
|
||||||
IncrementalIndex.SortedDimLookup dimLookup = accessor.getDimLookup();
|
if (!capabilities.hasBitmapIndexes()) {
|
||||||
final int id = dimLookup.getUnsortedIdFromSortedId(index);
|
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||||
if (id < 0 || id >= dimLookup.size()) {
|
}
|
||||||
|
|
||||||
|
final int id = (Integer) indexer.getUnsortedEncodedValueFromSorted(index);
|
||||||
|
if (id < 0 || id >= indexer.getCardinality()) {
|
||||||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -319,27 +261,6 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
return index.getCapabilities(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
|
static class BitmapIndexedInts implements IndexedInts
|
||||||
{
|
{
|
||||||
|
|
||||||
|
@ -407,4 +328,10 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
{
|
{
|
||||||
return index.getMetadata();
|
return index.getMetadata();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, DimensionHandler> getDimensionHandlers()
|
||||||
|
{
|
||||||
|
return index.getDimensionHandlers();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,16 +20,15 @@
|
||||||
package io.druid.segment.incremental;
|
package io.druid.segment.incremental;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Predicate;
|
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterators;
|
import com.google.common.collect.Iterators;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.primitives.Ints;
|
|
||||||
import com.metamx.common.guava.Sequence;
|
import com.metamx.common.guava.Sequence;
|
||||||
import com.metamx.common.guava.Sequences;
|
import com.metamx.common.guava.Sequences;
|
||||||
import io.druid.granularity.QueryGranularity;
|
import io.druid.granularity.QueryGranularity;
|
||||||
import io.druid.query.QueryInterruptedException;
|
import io.druid.query.QueryInterruptedException;
|
||||||
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.filter.DruidLongPredicate;
|
import io.druid.query.filter.DruidLongPredicate;
|
||||||
|
@ -39,6 +38,8 @@ import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.query.filter.ValueMatcherFactory;
|
import io.druid.query.filter.ValueMatcherFactory;
|
||||||
import io.druid.segment.Capabilities;
|
import io.druid.segment.Capabilities;
|
||||||
import io.druid.segment.Cursor;
|
import io.druid.segment.Cursor;
|
||||||
|
import io.druid.segment.DimensionHandler;
|
||||||
|
import io.druid.segment.DimensionIndexer;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
import io.druid.segment.FloatColumnSelector;
|
import io.druid.segment.FloatColumnSelector;
|
||||||
import io.druid.segment.LongColumnSelector;
|
import io.druid.segment.LongColumnSelector;
|
||||||
|
@ -51,7 +52,6 @@ import io.druid.segment.column.Column;
|
||||||
import io.druid.segment.column.ColumnCapabilities;
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
|
||||||
import io.druid.segment.data.ListIndexed;
|
import io.druid.segment.data.ListIndexed;
|
||||||
import io.druid.segment.filter.BooleanValueMatcher;
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
import io.druid.segment.filter.Filters;
|
import io.druid.segment.filter.Filters;
|
||||||
|
@ -59,11 +59,7 @@ import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -111,11 +107,14 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
if (dimension.equals(Column.TIME_COLUMN_NAME)) {
|
if (dimension.equals(Column.TIME_COLUMN_NAME)) {
|
||||||
return Integer.MAX_VALUE;
|
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 0;
|
||||||
}
|
}
|
||||||
return dimDim.size();
|
|
||||||
|
DimensionIndexer indexer = index.getDimension(dimension).getIndexer();
|
||||||
|
return indexer.getCardinality();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -139,15 +138,25 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
@Override
|
@Override
|
||||||
public Comparable getMinValue(String column)
|
public Comparable getMinValue(String column)
|
||||||
{
|
{
|
||||||
IncrementalIndex.DimDim dimDim = index.getDimensionValues(column);
|
IncrementalIndex.DimensionDesc desc = index.getDimension(column);
|
||||||
return dimDim == null ? null : dimDim.getMinValue();
|
if (desc == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
DimensionIndexer indexer = desc.getIndexer();
|
||||||
|
return indexer.getMinValue();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Comparable getMaxValue(String column)
|
public Comparable getMaxValue(String column)
|
||||||
{
|
{
|
||||||
IncrementalIndex.DimDim dimDim = index.getDimensionValues(column);
|
IncrementalIndex.DimensionDesc desc = index.getDimension(column);
|
||||||
return dimDim == null ? null : dimDim.getMaxValue();
|
if (desc == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
DimensionIndexer indexer = desc.getIndexer();
|
||||||
|
return indexer.getMaxValue();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -162,6 +171,12 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
return index.getCapabilities(column);
|
return index.getCapabilities(column);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, DimensionHandler> getDimensionHandlers()
|
||||||
|
{
|
||||||
|
return index.getDimensionHandlers();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getColumnTypeName(String column)
|
public String getColumnTypeName(String column)
|
||||||
{
|
{
|
||||||
|
@ -176,7 +191,12 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Cursor> makeCursors(final Filter filter, final Interval interval, final QueryGranularity gran, final boolean descending)
|
public Sequence<Cursor> makeCursors(
|
||||||
|
final Filter filter,
|
||||||
|
final Interval interval,
|
||||||
|
final QueryGranularity gran,
|
||||||
|
final boolean descending
|
||||||
|
)
|
||||||
{
|
{
|
||||||
if (index.isEmpty()) {
|
if (index.isEmpty()) {
|
||||||
return Sequences.empty();
|
return Sequences.empty();
|
||||||
|
@ -320,124 +340,42 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
public DimensionSelector makeDimensionSelector(
|
public DimensionSelector makeDimensionSelector(
|
||||||
DimensionSpec dimensionSpec
|
DimensionSpec dimensionSpec
|
||||||
)
|
)
|
||||||
{
|
|
||||||
return dimensionSpec.decorate(makeDimensionSelectorUndecorated(dimensionSpec));
|
|
||||||
}
|
|
||||||
|
|
||||||
private DimensionSelector makeDimensionSelectorUndecorated(
|
|
||||||
DimensionSpec dimensionSpec
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
final String dimension = dimensionSpec.getDimension();
|
final String dimension = dimensionSpec.getDimension();
|
||||||
final ExtractionFn extractionFn = dimensionSpec.getExtractionFn();
|
final ExtractionFn extractionFn = dimensionSpec.getExtractionFn();
|
||||||
|
|
||||||
if (dimension.equals(Column.TIME_COLUMN_NAME)) {
|
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) {
|
if (dimensionDesc == null) {
|
||||||
return NULL_DIMENSION_SELECTOR;
|
return dimensionSpec.decorate(NULL_DIMENSION_SELECTOR);
|
||||||
}
|
}
|
||||||
|
|
||||||
final int dimIndex = dimensionDesc.getIndex();
|
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||||
final IncrementalIndex.DimDim dimValLookup = dimensionDesc.getValues();
|
return dimensionSpec.decorate((DimensionSelector) indexer.makeColumnValueSelector(dimensionSpec, currEntry, dimensionDesc));
|
||||||
|
|
||||||
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<Integer> 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<Integer> 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<Integer> 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);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FloatColumnSelector makeFloatColumnSelector(String columnName)
|
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);
|
final Integer metricIndexInt = index.getMetricIndex(columnName);
|
||||||
if (metricIndexInt == null) {
|
if (metricIndexInt == null) {
|
||||||
return new FloatColumnSelector()
|
return new FloatColumnSelector()
|
||||||
|
@ -474,6 +412,18 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
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);
|
final Integer metricIndexInt = index.getMetricIndex(columnName);
|
||||||
if (metricIndexInt == null) {
|
if (metricIndexInt == null) {
|
||||||
return new LongColumnSelector()
|
return new LongColumnSelector()
|
||||||
|
@ -549,7 +499,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
if (dimensionDesc != null) {
|
if (dimensionDesc != null) {
|
||||||
|
|
||||||
final int dimensionIndex = dimensionDesc.getIndex();
|
final int dimensionIndex = dimensionDesc.getIndex();
|
||||||
final IncrementalIndex.DimDim dimDim = dimensionDesc.getValues();
|
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||||
|
final ColumnCapabilities capabilities = dimensionDesc.getCapabilities();
|
||||||
|
|
||||||
return new ObjectColumnSelector<Object>()
|
return new ObjectColumnSelector<Object>()
|
||||||
{
|
{
|
||||||
|
@ -567,22 +518,12 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
int[][] dims = key.getDims();
|
Object[] dims = key.getDims();
|
||||||
if (dimensionIndex >= dims.length) {
|
if (dimensionIndex >= dims.length) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
final int[] dimIdx = dims[dimensionIndex];
|
Object dimVals = indexer.convertUnsortedEncodedArrayToActualArrayOrList(dims[dimensionIndex], false);
|
||||||
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]);
|
|
||||||
}
|
|
||||||
return dimVals;
|
return dimVals;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
@ -617,7 +558,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
: filter.makeMatcher(new CursorAndEntryHolderValueMatcherFactory(cursor, holder));
|
: filter.makeMatcher(new CursorAndEntryHolderValueMatcherFactory(cursor, holder));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class EntryHolder
|
public static class EntryHolder
|
||||||
{
|
{
|
||||||
Map.Entry<IncrementalIndex.TimeAndDims, Integer> currEntry = null;
|
Map.Entry<IncrementalIndex.TimeAndDims, Integer> currEntry = null;
|
||||||
|
|
||||||
|
@ -658,96 +599,56 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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);
|
IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimension);
|
||||||
if (dimensionDesc == null) {
|
if (dimensionDesc == null) {
|
||||||
return new BooleanValueMatcher(isComparableNullOrEmpty(value));
|
// filtering on long metrics and __time is supported as well
|
||||||
}
|
final Integer metricIndexInt = index.getMetricIndex(dimension);
|
||||||
final int dimIndex = dimensionDesc.getIndex();
|
if (metricIndexInt != null || dimension.equals(Column.TIME_COLUMN_NAME)) {
|
||||||
final IncrementalIndex.DimDim dimDim = dimensionDesc.getValues();
|
ValueType type = getTypeForDimension(dimension);
|
||||||
|
switch (type) {
|
||||||
final Integer id = dimDim.getId(value);
|
case LONG:
|
||||||
if (id == null) {
|
return Filters.getLongValueMatcher(cursor.makeLongColumnSelector(dimension), originalValue);
|
||||||
if (isComparableNullOrEmpty(value)) {
|
default:
|
||||||
return new ValueMatcher()
|
return new BooleanValueMatcher(isComparableNullOrEmpty(originalValue));
|
||||||
{
|
|
||||||
@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);
|
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
return Ints.indexOf(dims[dimIndex], id) >= 0;
|
return new BooleanValueMatcher(isComparableNullOrEmpty(originalValue));
|
||||||
}
|
}
|
||||||
};
|
} else {
|
||||||
|
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||||
|
final int dimIndex = dimensionDesc.getIndex();
|
||||||
|
return indexer.makeIndexingValueMatcher(originalValue, holder, dimIndex);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ValueMatcher makeValueMatcher(String dimension, final DruidPredicateFactory predicateFactory)
|
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<String> predicate)
|
|
||||||
{
|
{
|
||||||
IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimension);
|
IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimension);
|
||||||
if (dimensionDesc == null) {
|
if (dimensionDesc == null) {
|
||||||
return new BooleanValueMatcher(predicate.apply(null));
|
// filtering on long metrics and __time is supported as well
|
||||||
}
|
final Integer metricIndexInt = index.getMetricIndex(dimension);
|
||||||
final int dimIndex = dimensionDesc.getIndex();
|
if (metricIndexInt != null || dimension.equals(Column.TIME_COLUMN_NAME)) {
|
||||||
final IncrementalIndex.DimDim dimDim = dimensionDesc.getValues();
|
ValueType type = getTypeForDimension(dimension);
|
||||||
|
switch (type) {
|
||||||
return new ValueMatcher()
|
case LONG:
|
||||||
{
|
return makeLongValueMatcher(dimension, predicateFactory.makeLongPredicate());
|
||||||
@Override
|
default:
|
||||||
public boolean matches()
|
return new BooleanValueMatcher(predicateFactory.makeStringPredicate().apply(null));
|
||||||
{
|
|
||||||
int[][] dims = holder.getKey().getDims();
|
|
||||||
if (dimIndex >= dims.length || dims[dimIndex] == null) {
|
|
||||||
return predicate.apply(null);
|
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
for (int dimVal : dims[dimIndex]) {
|
return new BooleanValueMatcher(predicateFactory.makeStringPredicate().apply(null));
|
||||||
if (predicate.apply((String) dimDim.getValue(dimVal))) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
};
|
} 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)
|
private ValueMatcher makeLongValueMatcher(String dimension, DruidLongPredicate predicate)
|
||||||
{
|
{
|
||||||
return Filters.getLongPredicateMatcher(cursor.makeLongColumnSelector(dimension), predicate);
|
return Filters.getLongPredicateMatcher(cursor.makeLongColumnSelector(dimension), predicate);
|
||||||
|
|
|
@ -79,7 +79,7 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
|
||||||
this.maxRowCount = maxRowCount;
|
this.maxRowCount = maxRowCount;
|
||||||
this.bufferPool = bufferPool;
|
this.bufferPool = bufferPool;
|
||||||
|
|
||||||
this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator())
|
this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions())
|
||||||
: new PlainFactsHolder(sortFacts);
|
: new PlainFactsHolder(sortFacts);
|
||||||
|
|
||||||
//check that stupid pool gives buffers that can hold at least one row's aggregators
|
//check that stupid pool gives buffers that can hold at least one row's aggregators
|
||||||
|
@ -163,12 +163,6 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
|
||||||
return facts;
|
return facts;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected DimDim makeDimDim(String dimension, Object lock)
|
|
||||||
{
|
|
||||||
return new OnheapIncrementalIndex.OnHeapDimDim(lock);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected BufferAggregator[] initAggs(
|
protected BufferAggregator[] initAggs(
|
||||||
AggregatorFactory[] metrics, Supplier<InputRow> rowSupplier, boolean deserializeComplexMetrics
|
AggregatorFactory[] metrics, Supplier<InputRow> rowSupplier, boolean deserializeComplexMetrics
|
||||||
|
@ -183,7 +177,8 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
|
||||||
ColumnSelectorFactory columnSelectorFactory = makeColumnSelectorFactory(
|
ColumnSelectorFactory columnSelectorFactory = makeColumnSelectorFactory(
|
||||||
agg,
|
agg,
|
||||||
rowSupplier,
|
rowSupplier,
|
||||||
deserializeComplexMetrics
|
deserializeComplexMetrics,
|
||||||
|
getColumnCapabilities()
|
||||||
);
|
);
|
||||||
|
|
||||||
selectors.put(
|
selectors.put(
|
||||||
|
@ -234,7 +229,7 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
|
||||||
for (int i = 0; i < metrics.length; i++) {
|
for (int i = 0; i < metrics.length; i++) {
|
||||||
final AggregatorFactory agg = metrics[i];
|
final AggregatorFactory agg = metrics[i];
|
||||||
getAggs()[i] = agg.factorizeBuffered(
|
getAggs()[i] = agg.factorizeBuffered(
|
||||||
makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics)
|
makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics, getColumnCapabilities())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
rowContainer.set(null);
|
rowContainer.set(null);
|
||||||
|
|
|
@ -67,7 +67,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
|
||||||
super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions);
|
super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions);
|
||||||
this.maxRowCount = maxRowCount;
|
this.maxRowCount = maxRowCount;
|
||||||
|
|
||||||
this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator())
|
this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions())
|
||||||
: new PlainFactsHolder(sortFacts);
|
: new PlainFactsHolder(sortFacts);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -146,12 +146,6 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
|
||||||
return facts;
|
return facts;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected DimDim makeDimDim(String dimension, Object lock)
|
|
||||||
{
|
|
||||||
return new OnHeapDimDim(lock);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Aggregator[] initAggs(
|
protected Aggregator[] initAggs(
|
||||||
AggregatorFactory[] metrics, Supplier<InputRow> rowSupplier, boolean deserializeComplexMetrics
|
AggregatorFactory[] metrics, Supplier<InputRow> rowSupplier, boolean deserializeComplexMetrics
|
||||||
|
@ -161,7 +155,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
|
||||||
for (AggregatorFactory agg : metrics) {
|
for (AggregatorFactory agg : metrics) {
|
||||||
selectors.put(
|
selectors.put(
|
||||||
agg.getName(),
|
agg.getName(),
|
||||||
new ObjectCachingColumnSelectorFactory(makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics))
|
new ObjectCachingColumnSelectorFactory(makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics, getColumnCapabilities()))
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -335,133 +329,6 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static class OnHeapDimDim<T extends Comparable<? super T>> implements DimDim<T>
|
|
||||||
{
|
|
||||||
private final Map<T, Integer> valueToId = Maps.newHashMap();
|
|
||||||
private T minValue = null;
|
|
||||||
private T maxValue = null;
|
|
||||||
|
|
||||||
private final List<T> 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<T extends Comparable<? super T>> implements SortedDimLookup<T>
|
|
||||||
{
|
|
||||||
private final List<T> sortedVals;
|
|
||||||
private final int[] idToIndex;
|
|
||||||
private final int[] indexToId;
|
|
||||||
|
|
||||||
public OnHeapDimLookup(List<T> idToValue, int length)
|
|
||||||
{
|
|
||||||
Map<T, Integer> 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.
|
// 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.
|
// 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.
|
// here its made thread safe to support the special case of groupBy where the multiple threads can add concurrently to the IncrementalIndex.
|
||||||
|
|
|
@ -29,6 +29,7 @@ import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
import com.google.common.util.concurrent.MoreExecutors;
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
|
import com.metamx.common.IAE;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.guava.MergeSequence;
|
import com.metamx.common.guava.MergeSequence;
|
||||||
import com.metamx.common.guava.Sequence;
|
import com.metamx.common.guava.Sequence;
|
||||||
|
@ -4785,12 +4786,14 @@ public class GroupByQueryRunnerTest
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
// v1 strategy would throw an exception for this because it calls AggregatorFactory.getRequiredColumns to get
|
// 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 for all fields to build the inner query result incremental index. In this case, quality is a string
|
||||||
// aggregator value type, parse exception occurs. In this case, quality is a string field but getRequiredColumn
|
// field but getRequiredColumn() returned a Cardinality aggregator for it, which has type hyperUnique.
|
||||||
// returned a Cardinality aggregator for it, which has type hyperUnique. Since this is a complex type, no converter
|
// The "quality" column is interpreted as a dimension because it appears in the dimension list of the
|
||||||
// is found for it and NullPointerException occurs when it tries to use the converter.
|
// 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)) {
|
if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) {
|
||||||
expectedException.expect(NullPointerException.class);
|
expectedException.expect(IAE.class);
|
||||||
GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
|
GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
|
||||||
} else {
|
} else {
|
||||||
List<Row> expectedResults = Arrays.asList(
|
List<Row> expectedResults = Arrays.asList(
|
||||||
|
@ -4884,11 +4887,16 @@ public class GroupByQueryRunnerTest
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
// v1 strategy would throw an exception for this because it calls AggregatorFactory.getRequiredColumns to get
|
// 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 for all fields to build the inner query result incremental index. In this case, market is a string
|
||||||
// aggregator value type, parse exception occurs. In this case, market is a string field but getRequiredColumn
|
// field but getRequiredColumn() returned a Javascript aggregator for it, which has type float.
|
||||||
// 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)) {
|
if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) {
|
||||||
expectedException.expect(ParseException.class);
|
expectedException.expect(IAE.class);
|
||||||
GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
|
GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
|
||||||
} else {
|
} else {
|
||||||
List<Row> expectedResults = Arrays.asList(
|
List<Row> expectedResults = Arrays.asList(
|
||||||
|
|
|
@ -30,6 +30,7 @@ import com.google.common.primitives.Ints;
|
||||||
import com.metamx.collections.bitmap.RoaringBitmapFactory;
|
import com.metamx.collections.bitmap.RoaringBitmapFactory;
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
|
import com.metamx.common.io.smoosh.SmooshedFileMapper;
|
||||||
import io.druid.data.input.MapBasedInputRow;
|
import io.druid.data.input.MapBasedInputRow;
|
||||||
import io.druid.data.input.impl.DimensionsSpec;
|
import io.druid.data.input.impl.DimensionsSpec;
|
||||||
import io.druid.granularity.QueryGranularities;
|
import io.druid.granularity.QueryGranularities;
|
||||||
|
@ -69,6 +70,7 @@ import java.nio.IntBuffer;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -1986,6 +1988,50 @@ public class IndexMergerTest
|
||||||
Assert.assertEquals(ImmutableSet.of("A", "B", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics()));
|
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.<String, Object>of("d1", "a", "d2", "", "A", 1)
|
||||||
|
));
|
||||||
|
|
||||||
|
index1.add(new MapBasedInputRow(
|
||||||
|
1L,
|
||||||
|
Lists.newArrayList("d1", "d2"),
|
||||||
|
ImmutableMap.<String, Object>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<String> expectedColumnNames = Arrays.asList("A", "d1");
|
||||||
|
List<String> actualColumnNames = Lists.newArrayList(index.getColumnNames());
|
||||||
|
Collections.sort(expectedColumnNames);
|
||||||
|
Collections.sort(actualColumnNames);
|
||||||
|
Assert.assertEquals(expectedColumnNames, actualColumnNames);
|
||||||
|
|
||||||
|
SmooshedFileMapper sfm = closer.closeLater(SmooshedFileMapper.load(tempDir));
|
||||||
|
List<String> expectedFilenames = Arrays.asList("A", "__time", "d1", "index.drd", "metadata.drd");
|
||||||
|
List<String> actualFilenames = new ArrayList<>(sfm.getInternalFilenames());
|
||||||
|
Collections.sort(expectedFilenames);
|
||||||
|
Collections.sort(actualFilenames);
|
||||||
|
Assert.assertEquals(expectedFilenames, actualFilenames);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
private IncrementalIndex getIndexD3() throws Exception
|
private IncrementalIndex getIndexD3() throws Exception
|
||||||
{
|
{
|
||||||
IncrementalIndex toPersist1 = new OnheapIncrementalIndex(
|
IncrementalIndex toPersist1 = new OnheapIncrementalIndex(
|
||||||
|
|
|
@ -19,7 +19,9 @@
|
||||||
|
|
||||||
package io.druid.segment.filter;
|
package io.druid.segment.filter;
|
||||||
|
|
||||||
|
import io.druid.segment.DimensionHandler;
|
||||||
import io.druid.segment.Rowboat;
|
import io.druid.segment.Rowboat;
|
||||||
|
import io.druid.segment.StringDimensionHandler;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -28,21 +30,31 @@ import org.junit.Test;
|
||||||
*/
|
*/
|
||||||
public class RowboatTest
|
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
|
@Test
|
||||||
public void testRowboatCompare()
|
public void testRowboatCompare()
|
||||||
{
|
{
|
||||||
Rowboat rb1 = new Rowboat(12345L, new int[][]{new int[]{1}, new int[]{2}}, new Object[]{new Integer(7)}, 5);
|
DimensionHandler[] handlers = getDefaultHandlers(3);
|
||||||
Rowboat rb2 = new Rowboat(12345L, new int[][]{new int[]{1}, new int[]{2}}, new Object[]{new Integer(7)}, 5);
|
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));
|
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));
|
Assert.assertNotEquals(0, rb1.compareTo(rb3));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testBiggerCompare()
|
public void testBiggerCompare()
|
||||||
{
|
{
|
||||||
|
DimensionHandler[] handlers = getDefaultHandlers(14);
|
||||||
|
|
||||||
Rowboat rb1 = new Rowboat(
|
Rowboat rb1 = new Rowboat(
|
||||||
0,
|
0,
|
||||||
new int[][]{
|
new int[][]{
|
||||||
|
@ -62,7 +74,8 @@ public class RowboatTest
|
||||||
new int[]{0}
|
new int[]{0}
|
||||||
},
|
},
|
||||||
new Object[]{1.0, 47.0, "someMetric"},
|
new Object[]{1.0, 47.0, "someMetric"},
|
||||||
0
|
0,
|
||||||
|
handlers
|
||||||
);
|
);
|
||||||
|
|
||||||
Rowboat rb2 = new Rowboat(
|
Rowboat rb2 = new Rowboat(
|
||||||
|
@ -84,7 +97,8 @@ public class RowboatTest
|
||||||
new int[]{0}
|
new int[]{0}
|
||||||
},
|
},
|
||||||
new Object[]{1.0, 47.0, "someMetric"},
|
new Object[]{1.0, 47.0, "someMetric"},
|
||||||
0
|
0,
|
||||||
|
handlers
|
||||||
);
|
);
|
||||||
|
|
||||||
Assert.assertNotEquals(0, rb1.compareTo(rb2));
|
Assert.assertNotEquals(0, rb1.compareTo(rb2));
|
||||||
|
@ -93,18 +107,22 @@ public class RowboatTest
|
||||||
@Test
|
@Test
|
||||||
public void testToString()
|
public void testToString()
|
||||||
{
|
{
|
||||||
|
DimensionHandler[] handlers = getDefaultHandlers(2);
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
"Rowboat{timestamp=1970-01-01T00:00:00.000Z, dims=[[1], [2]], metrics=[someMetric], comprisedRows={}}",
|
"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
|
@Test
|
||||||
public void testLotsONullString()
|
public void testLotsONullString()
|
||||||
{
|
{
|
||||||
|
DimensionHandler[] handlers = getDefaultHandlers(0);
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
"Rowboat{timestamp=1970-01-01T00:00:00.000Z, dims=null, metrics=null, comprisedRows={}}",
|
"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()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,6 +27,7 @@ import com.metamx.common.ISE;
|
||||||
import io.druid.collections.StupidPool;
|
import io.druid.collections.StupidPool;
|
||||||
import io.druid.data.input.MapBasedInputRow;
|
import io.druid.data.input.MapBasedInputRow;
|
||||||
import io.druid.data.input.Row;
|
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.DimensionsSpec;
|
||||||
import io.druid.data.input.impl.FloatDimensionSchema;
|
import io.druid.data.input.impl.FloatDimensionSchema;
|
||||||
import io.druid.data.input.impl.LongDimensionSchema;
|
import io.druid.data.input.impl.LongDimensionSchema;
|
||||||
|
@ -75,10 +76,10 @@ public class IncrementalIndexTest
|
||||||
public static Collection<?> constructorFeeder() throws IOException
|
public static Collection<?> constructorFeeder() throws IOException
|
||||||
{
|
{
|
||||||
DimensionsSpec dimensions = new DimensionsSpec(
|
DimensionsSpec dimensions = new DimensionsSpec(
|
||||||
Arrays.asList(
|
Arrays.<DimensionSchema>asList(
|
||||||
new StringDimensionSchema("string"),
|
new StringDimensionSchema("string"),
|
||||||
new FloatDimensionSchema("float"),
|
new StringDimensionSchema("float"),
|
||||||
new LongDimensionSchema("long")
|
new StringDimensionSchema("long")
|
||||||
), null, null
|
), null, null
|
||||||
);
|
);
|
||||||
AggregatorFactory[] metrics = {
|
AggregatorFactory[] metrics = {
|
||||||
|
@ -216,8 +217,8 @@ public class IncrementalIndexTest
|
||||||
Row row = index.iterator().next();
|
Row row = index.iterator().next();
|
||||||
|
|
||||||
Assert.assertEquals(Arrays.asList(new String[]{"", "", "A"}), row.getRaw("string"));
|
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 String[]{"", "", String.valueOf(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(Long.MIN_VALUE)}), row.getRaw("long"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -158,7 +158,7 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark
|
||||||
for (int i = 0; i < metrics.length; i++) {
|
for (int i = 0; i < metrics.length; i++) {
|
||||||
final AggregatorFactory agg = metrics[i];
|
final AggregatorFactory agg = metrics[i];
|
||||||
aggs[i] = agg.factorize(
|
aggs[i] = agg.factorize(
|
||||||
makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics)
|
makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics, null)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
Integer rowIndex;
|
Integer rowIndex;
|
||||||
|
|
|
@ -24,12 +24,14 @@ import com.google.common.base.Predicate;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Iterators;
|
import com.google.common.collect.Iterators;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import io.druid.data.input.InputRow;
|
import io.druid.data.input.InputRow;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.segment.QueryableIndex;
|
import io.druid.segment.QueryableIndex;
|
||||||
|
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
import io.druid.segment.incremental.IndexSizeExceededException;
|
import io.druid.segment.incremental.IndexSizeExceededException;
|
||||||
|
@ -45,6 +47,7 @@ import java.util.Arrays;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.LinkedHashSet;
|
import java.util.LinkedHashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
|
@ -262,16 +265,20 @@ public class Sink implements Iterable<FireHydrant>
|
||||||
FireHydrant lastHydrant = hydrants.get(numHydrants - 1);
|
FireHydrant lastHydrant = hydrants.get(numHydrants - 1);
|
||||||
newCount = lastHydrant.getCount() + 1;
|
newCount = lastHydrant.getCount() + 1;
|
||||||
if (!indexSchema.getDimensionsSpec().hasCustomDimensions()) {
|
if (!indexSchema.getDimensionsSpec().hasCustomDimensions()) {
|
||||||
|
Map<String, ColumnCapabilitiesImpl> oldCapabilities;
|
||||||
if (lastHydrant.hasSwapped()) {
|
if (lastHydrant.hasSwapped()) {
|
||||||
|
oldCapabilities = Maps.newHashMap();
|
||||||
QueryableIndex oldIndex = lastHydrant.getSegment().asQueryableIndex();
|
QueryableIndex oldIndex = lastHydrant.getSegment().asQueryableIndex();
|
||||||
for (String dim : oldIndex.getAvailableDimensions()) {
|
for (String dim : oldIndex.getAvailableDimensions()) {
|
||||||
dimOrder.add(dim);
|
dimOrder.add(dim);
|
||||||
|
oldCapabilities.put(dim, (ColumnCapabilitiesImpl) oldIndex.getColumn(dim).getCapabilities());
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
IncrementalIndex oldIndex = lastHydrant.getIndex();
|
IncrementalIndex oldIndex = lastHydrant.getIndex();
|
||||||
dimOrder.addAll(oldIndex.getDimensionOrder());
|
dimOrder.addAll(oldIndex.getDimensionOrder());
|
||||||
|
oldCapabilities = oldIndex.getColumnCapabilities();
|
||||||
}
|
}
|
||||||
newIndex.loadDimensionIterable(dimOrder);
|
newIndex.loadDimensionIterable(dimOrder, oldCapabilities);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
currHydrant = new FireHydrant(newIndex, newCount, getSegment().getIdentifier());
|
currHydrant = new FireHydrant(newIndex, newCount, getSegment().getIdentifier());
|
||||||
|
|
Loading…
Reference in New Issue