mirror of https://github.com/apache/druid.git
precursor changes for nested columns to minimize files changed (#12714)
* precursor changes for nested columns to minimize files changed * inspection fix * visibility * adjustment * unecessary change
This commit is contained in:
parent
d30efb1c1e
commit
48731710fb
|
@ -187,7 +187,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
|
||||||
dimConversions.set(i, dictionaryMergeIterator.conversions[i]);
|
dimConversions.set(i, dictionaryMergeIterator.conversions[i]);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
cardinality = dictionaryMergeIterator.counter;
|
cardinality = dictionaryMergeIterator.getCardinality();
|
||||||
} else if (numMergeIndex == 1) {
|
} else if (numMergeIndex == 1) {
|
||||||
writeDictionary(dimValueLookup);
|
writeDictionary(dimValueLookup);
|
||||||
cardinality = dimValueLookup.size();
|
cardinality = dimValueLookup.size();
|
||||||
|
|
|
@ -124,6 +124,11 @@ public class DictionaryMergingIterator<T extends Comparable<T>> implements Close
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public int getCardinality()
|
||||||
|
{
|
||||||
|
return counter;
|
||||||
|
}
|
||||||
|
|
||||||
protected PeekingIterator<T> transformIndexedIterator(Indexed<T> indexed)
|
protected PeekingIterator<T> transformIndexedIterator(Indexed<T> indexed)
|
||||||
{
|
{
|
||||||
return Iterators.peekingIterator(
|
return Iterators.peekingIterator(
|
||||||
|
|
|
@ -42,6 +42,11 @@ public class ColumnBuilder
|
||||||
@Nullable
|
@Nullable
|
||||||
private SmooshedFileMapper fileMapper = null;
|
private SmooshedFileMapper fileMapper = null;
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
public ColumnCapabilitiesImpl getCapabilitiesBuilder()
|
||||||
|
{
|
||||||
|
return capabilitiesBuilder;
|
||||||
|
}
|
||||||
|
|
||||||
public ColumnBuilder setFileMapper(SmooshedFileMapper fileMapper)
|
public ColumnBuilder setFileMapper(SmooshedFileMapper fileMapper)
|
||||||
{
|
{
|
||||||
|
|
|
@ -190,7 +190,7 @@ public enum CompressionStrategy
|
||||||
* <p>
|
* <p>
|
||||||
* If the allocated buffer is a direct buffer, it should be registered to be freed with the given Closer.
|
* If the allocated buffer is a direct buffer, it should be registered to be freed with the given Closer.
|
||||||
*/
|
*/
|
||||||
ByteBuffer allocateInBuffer(int inputSize, Closer closer)
|
public ByteBuffer allocateInBuffer(int inputSize, Closer closer)
|
||||||
{
|
{
|
||||||
return ByteBuffer.allocate(inputSize);
|
return ByteBuffer.allocate(inputSize);
|
||||||
}
|
}
|
||||||
|
@ -203,7 +203,7 @@ public enum CompressionStrategy
|
||||||
* <p>
|
* <p>
|
||||||
* If the allocated buffer is a direct buffer, it should be registered to be freed with the given Closer.
|
* If the allocated buffer is a direct buffer, it should be registered to be freed with the given Closer.
|
||||||
*/
|
*/
|
||||||
abstract ByteBuffer allocateOutBuffer(int inputSize, Closer closer);
|
public abstract ByteBuffer allocateOutBuffer(int inputSize, Closer closer);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a ByteBuffer with compressed contents of in between it's position and limit. It may be the provided out
|
* Returns a ByteBuffer with compressed contents of in between it's position and limit. It may be the provided out
|
||||||
|
@ -221,7 +221,7 @@ public enum CompressionStrategy
|
||||||
private static final UncompressedCompressor DEFAULT_COMPRESSOR = new UncompressedCompressor();
|
private static final UncompressedCompressor DEFAULT_COMPRESSOR = new UncompressedCompressor();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
ByteBuffer allocateOutBuffer(int inputSize, Closer closer)
|
public ByteBuffer allocateOutBuffer(int inputSize, Closer closer)
|
||||||
{
|
{
|
||||||
return ByteBuffer.allocate(inputSize);
|
return ByteBuffer.allocate(inputSize);
|
||||||
}
|
}
|
||||||
|
@ -333,7 +333,7 @@ public enum CompressionStrategy
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
ByteBuffer allocateInBuffer(int inputSize, Closer closer)
|
public ByteBuffer allocateInBuffer(int inputSize, Closer closer)
|
||||||
{
|
{
|
||||||
ByteBuffer inBuffer = ByteBuffer.allocateDirect(inputSize);
|
ByteBuffer inBuffer = ByteBuffer.allocateDirect(inputSize);
|
||||||
closer.register(() -> ByteBufferUtils.free(inBuffer));
|
closer.register(() -> ByteBufferUtils.free(inBuffer));
|
||||||
|
@ -341,7 +341,7 @@ public enum CompressionStrategy
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
ByteBuffer allocateOutBuffer(int inputSize, Closer closer)
|
public ByteBuffer allocateOutBuffer(int inputSize, Closer closer)
|
||||||
{
|
{
|
||||||
ByteBuffer outBuffer = ByteBuffer.allocateDirect(LZ4_HIGH.maxCompressedLength(inputSize));
|
ByteBuffer outBuffer = ByteBuffer.allocateDirect(LZ4_HIGH.maxCompressedLength(inputSize));
|
||||||
closer.register(() -> ByteBufferUtils.free(outBuffer));
|
closer.register(() -> ByteBufferUtils.free(outBuffer));
|
||||||
|
@ -365,7 +365,7 @@ public enum CompressionStrategy
|
||||||
private static final ZstdCompressor DEFAULT_COMPRESSOR = new ZstdCompressor();
|
private static final ZstdCompressor DEFAULT_COMPRESSOR = new ZstdCompressor();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
ByteBuffer allocateInBuffer(int inputSize, Closer closer)
|
public ByteBuffer allocateInBuffer(int inputSize, Closer closer)
|
||||||
{
|
{
|
||||||
ByteBuffer inBuffer = ByteBuffer.allocateDirect(inputSize);
|
ByteBuffer inBuffer = ByteBuffer.allocateDirect(inputSize);
|
||||||
closer.register(() -> ByteBufferUtils.free(inBuffer));
|
closer.register(() -> ByteBufferUtils.free(inBuffer));
|
||||||
|
@ -373,7 +373,7 @@ public enum CompressionStrategy
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
ByteBuffer allocateOutBuffer(int inputSize, Closer closer)
|
public ByteBuffer allocateOutBuffer(int inputSize, Closer closer)
|
||||||
{
|
{
|
||||||
ByteBuffer outBuffer = ByteBuffer.allocateDirect((int) Zstd.compressBound(inputSize));
|
ByteBuffer outBuffer = ByteBuffer.allocateDirect((int) Zstd.compressBound(inputSize));
|
||||||
closer.register(() -> ByteBufferUtils.free(outBuffer));
|
closer.register(() -> ByteBufferUtils.free(outBuffer));
|
||||||
|
|
|
@ -45,25 +45,6 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
private final IncrementalIndex index;
|
private final IncrementalIndex index;
|
||||||
private final Map<String, DimensionAccessor> accessors;
|
private final Map<String, DimensionAccessor> accessors;
|
||||||
|
|
||||||
private static class DimensionAccessor
|
|
||||||
{
|
|
||||||
private final IncrementalIndex.DimensionDesc dimensionDesc;
|
|
||||||
@Nullable
|
|
||||||
private final MutableBitmap[] invertedIndexes;
|
|
||||||
private final DimensionIndexer indexer;
|
|
||||||
|
|
||||||
public DimensionAccessor(IncrementalIndex.DimensionDesc dimensionDesc)
|
|
||||||
{
|
|
||||||
this.dimensionDesc = dimensionDesc;
|
|
||||||
this.indexer = dimensionDesc.getIndexer();
|
|
||||||
if (dimensionDesc.getCapabilities().hasBitmapIndexes()) {
|
|
||||||
this.invertedIndexes = new MutableBitmap[indexer.getCardinality() + 1];
|
|
||||||
} else {
|
|
||||||
this.invertedIndexes = null;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public IncrementalIndexAdapter(Interval dataInterval, IncrementalIndex index, BitmapFactory bitmapFactory)
|
public IncrementalIndexAdapter(Interval dataInterval, IncrementalIndex index, BitmapFactory bitmapFactory)
|
||||||
{
|
{
|
||||||
this.dataInterval = dataInterval;
|
this.dataInterval = dataInterval;
|
||||||
|
@ -118,6 +99,12 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
public IncrementalIndex getIncrementalIndex()
|
||||||
|
{
|
||||||
|
return index;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Interval getDataInterval()
|
public Interval getDataInterval()
|
||||||
{
|
{
|
||||||
|
@ -190,6 +177,24 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
return new MutableBitmapValues(bitmapIndex);
|
return new MutableBitmapValues(bitmapIndex);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getMetricType(String metric)
|
||||||
|
{
|
||||||
|
return index.getMetricType(metric);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ColumnCapabilities getCapabilities(String column)
|
||||||
|
{
|
||||||
|
return index.getColumnCapabilities(column);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Metadata getMetadata()
|
||||||
|
{
|
||||||
|
return index.getMetadata();
|
||||||
|
}
|
||||||
|
|
||||||
static class MutableBitmapValues implements BitmapValues
|
static class MutableBitmapValues implements BitmapValues
|
||||||
{
|
{
|
||||||
private final MutableBitmap bitmapIndex;
|
private final MutableBitmap bitmapIndex;
|
||||||
|
@ -212,21 +217,22 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
private static class DimensionAccessor
|
||||||
public String getMetricType(String metric)
|
|
||||||
{
|
{
|
||||||
return index.getMetricType(metric);
|
private final IncrementalIndex.DimensionDesc dimensionDesc;
|
||||||
}
|
@Nullable
|
||||||
|
private final MutableBitmap[] invertedIndexes;
|
||||||
|
private final DimensionIndexer indexer;
|
||||||
|
|
||||||
@Override
|
public DimensionAccessor(IncrementalIndex.DimensionDesc dimensionDesc)
|
||||||
public ColumnCapabilities getCapabilities(String column)
|
|
||||||
{
|
{
|
||||||
return index.getColumnCapabilities(column);
|
this.dimensionDesc = dimensionDesc;
|
||||||
|
this.indexer = dimensionDesc.getIndexer();
|
||||||
|
if (dimensionDesc.getCapabilities().hasBitmapIndexes()) {
|
||||||
|
this.invertedIndexes = new MutableBitmap[indexer.getCardinality() + 1];
|
||||||
|
} else {
|
||||||
|
this.invertedIndexes = null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Metadata getMetadata()
|
|
||||||
{
|
|
||||||
return index.getMetadata();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -56,10 +56,10 @@ import java.nio.channels.WritableByteChannel;
|
||||||
|
|
||||||
public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
||||||
{
|
{
|
||||||
private static final int NO_FLAGS = 0;
|
public static final int NO_FLAGS = 0;
|
||||||
private static final int STARTING_FLAGS = Feature.NO_BITMAP_INDEX.getMask();
|
public static final int STARTING_FLAGS = Feature.NO_BITMAP_INDEX.getMask();
|
||||||
|
|
||||||
enum Feature
|
public enum Feature
|
||||||
{
|
{
|
||||||
MULTI_VALUE,
|
MULTI_VALUE,
|
||||||
MULTI_VALUE_V3,
|
MULTI_VALUE_V3,
|
||||||
|
@ -76,7 +76,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
enum VERSION
|
public enum VERSION
|
||||||
{
|
{
|
||||||
UNCOMPRESSED_SINGLE_VALUE, // 0x0
|
UNCOMPRESSED_SINGLE_VALUE, // 0x0
|
||||||
UNCOMPRESSED_MULTI_VALUE, // 0x1
|
UNCOMPRESSED_MULTI_VALUE, // 0x1
|
||||||
|
|
|
@ -30,7 +30,7 @@ import javax.annotation.Nullable;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
class ExpressionVectorInputBinding implements Expr.VectorInputBinding
|
public class ExpressionVectorInputBinding implements Expr.VectorInputBinding
|
||||||
{
|
{
|
||||||
private final Map<String, VectorValueSelector> numeric;
|
private final Map<String, VectorValueSelector> numeric;
|
||||||
private final Map<String, VectorObjectSelector> objects;
|
private final Map<String, VectorObjectSelector> objects;
|
||||||
|
|
|
@ -847,6 +847,11 @@ public class AggregationTestHelper implements Closeable
|
||||||
return results;
|
return results;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public IndexIO getIndexIO()
|
||||||
|
{
|
||||||
|
return indexIO;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException
|
public void close() throws IOException
|
||||||
{
|
{
|
||||||
|
|
|
@ -460,6 +460,13 @@ public class OperatorConversions
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
public OperatorBuilder operandTypeInference(SqlOperandTypeInference operandTypeInference)
|
||||||
|
{
|
||||||
|
this.operandTypeInference = operandTypeInference;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a {@link SqlFunction} from this builder.
|
* Creates a {@link SqlFunction} from this builder.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -116,9 +116,14 @@ public class CastOperatorConversion implements SqlOperatorConversion
|
||||||
} else if (SqlTypeName.DATETIME_TYPES.contains(fromType) && SqlTypeName.CHAR_TYPES.contains(toType)) {
|
} else if (SqlTypeName.DATETIME_TYPES.contains(fromType) && SqlTypeName.CHAR_TYPES.contains(toType)) {
|
||||||
return castDateTimeToChar(plannerContext, operandExpression, fromType, Calcites.getColumnTypeForRelDataType(rexNode.getType()));
|
return castDateTimeToChar(plannerContext, operandExpression, fromType, Calcites.getColumnTypeForRelDataType(rexNode.getType()));
|
||||||
} else {
|
} else {
|
||||||
// Handle other casts.
|
// Handle other casts. If either type is ANY, use the other type instead. If both are ANY, this means nulls
|
||||||
final ExprType fromExprType = EXPRESSION_TYPES.get(fromType);
|
// downstream, Druid will try its best
|
||||||
final ExprType toExprType = EXPRESSION_TYPES.get(toType);
|
final ExprType fromExprType = SqlTypeName.ANY.equals(fromType)
|
||||||
|
? EXPRESSION_TYPES.get(toType)
|
||||||
|
: EXPRESSION_TYPES.get(fromType);
|
||||||
|
final ExprType toExprType = SqlTypeName.ANY.equals(toType)
|
||||||
|
? EXPRESSION_TYPES.get(fromType)
|
||||||
|
: EXPRESSION_TYPES.get(toType);
|
||||||
|
|
||||||
if (fromExprType == null || toExprType == null) {
|
if (fromExprType == null || toExprType == null) {
|
||||||
// We have no runtime type for these SQL types.
|
// We have no runtime type for these SQL types.
|
||||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.druid.math.expr.Parser;
|
||||||
import org.apache.druid.segment.column.RowSignature;
|
import org.apache.druid.segment.column.RowSignature;
|
||||||
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||||
import org.apache.druid.sql.calcite.expression.Expressions;
|
import org.apache.druid.sql.calcite.expression.Expressions;
|
||||||
import org.apache.druid.sql.calcite.table.RowSignatures;
|
|
||||||
|
|
||||||
import java.math.BigDecimal;
|
import java.math.BigDecimal;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
@ -168,7 +167,7 @@ public class DruidRexExecutor implements RexExecutor
|
||||||
} else {
|
} else {
|
||||||
literal = rexBuilder.makeLiteral(Arrays.asList(exprResult.asArray()), constExp.getType(), true);
|
literal = rexBuilder.makeLiteral(Arrays.asList(exprResult.asArray()), constExp.getType(), true);
|
||||||
}
|
}
|
||||||
} else if (sqlTypeName == SqlTypeName.OTHER && constExp.getType() instanceof RowSignatures.ComplexSqlType) {
|
} else if (sqlTypeName == SqlTypeName.OTHER) {
|
||||||
// complex constant is not reducible, so just leave it as an expression
|
// complex constant is not reducible, so just leave it as an expression
|
||||||
literal = constExp;
|
literal = constExp;
|
||||||
} else {
|
} else {
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.druid.sql.calcite.rel;
|
package org.apache.druid.sql.calcite.rel;
|
||||||
|
|
||||||
import org.apache.calcite.rel.type.RelDataType;
|
import org.apache.calcite.rel.type.RelDataType;
|
||||||
|
import org.apache.calcite.sql.type.SqlTypeName;
|
||||||
import org.apache.druid.math.expr.ExprMacroTable;
|
import org.apache.druid.math.expr.ExprMacroTable;
|
||||||
import org.apache.druid.segment.VirtualColumn;
|
import org.apache.druid.segment.VirtualColumn;
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
|
@ -127,6 +128,13 @@ public class VirtualColumnRegistry
|
||||||
RelDataType typeHint
|
RelDataType typeHint
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
if (typeHint.getSqlTypeName() == SqlTypeName.OTHER && expression.getDruidType() != null) {
|
||||||
|
// fall back to druid type if sql type isn't very helpful
|
||||||
|
return getOrCreateVirtualColumnForExpression(
|
||||||
|
expression,
|
||||||
|
expression.getDruidType()
|
||||||
|
);
|
||||||
|
}
|
||||||
return getOrCreateVirtualColumnForExpression(
|
return getOrCreateVirtualColumnForExpression(
|
||||||
expression,
|
expression,
|
||||||
Calcites.getColumnTypeForRelDataType(typeHint)
|
Calcites.getColumnTypeForRelDataType(typeHint)
|
||||||
|
|
Loading…
Reference in New Issue