Spotbugs: NP_NONNULL_PARAM_VIOLATION (#8129)

This commit is contained in:
Fokko Driesprong 2019-08-02 18:20:22 +02:00 committed by Roman Leventov
parent 4bd3bad8ba
commit 91743eeebe
43 changed files with 116 additions and 75 deletions

View File

@ -60,8 +60,6 @@
<Bug pattern="NP_LOAD_OF_KNOWN_NULL_VALUE"/>
<Bug pattern="NP_METHOD_PARAMETER_TIGHTENS_ANNOTATION"/>
<Bug pattern="NP_NONNULL_FIELD_NOT_INITIALIZED_IN_CONSTRUCTOR"/>
<Bug pattern="NP_NONNULL_PARAM_VIOLATION"/>
<Bug pattern="NP_NONNULL_PARAM_VIOLATION"/>
<Bug pattern="NP_NONNULL_RETURN_VIOLATION"/>
<Bug pattern="NP_NULL_ON_SOME_PATH"/>
<Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"/>

View File

@ -23,10 +23,12 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.Pair;
import javax.annotation.Nullable;
public class SerializablePair<T1, T2> extends Pair<T1, T2>
{
@JsonCreator
public SerializablePair(@JsonProperty("lhs") T1 lhs, @JsonProperty("rhs") T2 rhs)
public SerializablePair(@JsonProperty("lhs") T1 lhs, @JsonProperty("rhs") @Nullable T2 rhs)
{
super(lhs, rhs);
}
@ -38,6 +40,7 @@ public class SerializablePair<T1, T2> extends Pair<T1, T2>
}
@JsonProperty
@Nullable
public T2 getRhs()
{
return rhs;

View File

@ -29,6 +29,8 @@ import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.aggregation.post.PostAggregatorIds;
import org.apache.druid.query.cache.CacheKeyBuilder;
import javax.annotation.Nullable;
import java.util.Comparator;
import java.util.HashSet;
import java.util.Map;
@ -39,13 +41,14 @@ public class SketchEstimatePostAggregator implements PostAggregator
private final String name;
private final PostAggregator field;
@Nullable
private final Integer errorBoundsStdDev;
@JsonCreator
public SketchEstimatePostAggregator(
@JsonProperty("name") String name,
@JsonProperty("field") PostAggregator field,
@JsonProperty("errorBoundsStdDev") Integer errorBoundsStdDev
@JsonProperty("errorBoundsStdDev") @Nullable Integer errorBoundsStdDev
)
{
this.name = Preconditions.checkNotNull(name, "name is null");
@ -56,9 +59,7 @@ public class SketchEstimatePostAggregator implements PostAggregator
@Override
public Set<String> getDependentFields()
{
Set<String> dependentFields = new HashSet<>();
dependentFields.addAll(field.getDependentFields());
return dependentFields;
return new HashSet<>(field.getDependentFields());
}
@Override
@ -111,6 +112,7 @@ public class SketchEstimatePostAggregator implements PostAggregator
return field;
}
@Nullable
@JsonProperty
public Integer getErrorBoundsStdDev()
{

View File

@ -23,6 +23,8 @@ import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Floats;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
@ -470,11 +472,6 @@ public class ApproximateHistogram
}
}
public ApproximateHistogram fold(ApproximateHistogram h)
{
return fold(h, null, null, null);
}
public ApproximateHistogram fold(ApproximateHistogram h, float[] mergedPositions, long[] mergedBins, float[] deltas)
{
if (size == 0) {
@ -496,7 +493,7 @@ public class ApproximateHistogram
*
* @return returns this histogram with h folded into it
*/
public ApproximateHistogram foldFast(ApproximateHistogram h, float[] mergedPositions, long[] mergedBins)
public ApproximateHistogram foldFast(ApproximateHistogram h, @Nullable float[] mergedPositions, @Nullable long[] mergedBins)
{
if (size == 0) {
return copy(h);
@ -612,7 +609,7 @@ public class ApproximateHistogram
return this;
}
protected ApproximateHistogram foldRule(ApproximateHistogram h, float[] mergedPositions, long[] mergedBins)
protected ApproximateHistogram foldRule(ApproximateHistogram h, @Nullable float[] mergedPositions, @Nullable long[] mergedBins)
{
// ruleCombine bins requires at least one bin
if (h.binCount == 0) {
@ -1079,8 +1076,8 @@ public class ApproximateHistogram
float[] rightPositions,
long[] rightBins,
float[] mergedPositions,
long[] mergedBins,
float[] deltas
@Nullable long[] mergedBins,
@Nullable float[] deltas
)
{
int i = 0;

View File

@ -144,7 +144,7 @@ public class ApproximateHistogramErrorBenchmark
for (int i = 0; i < numValues; ++i) {
tmp.offer(values[i]);
if ((i + 1) % numPerHist == 0) {
ah1.fold(tmp);
ah1.fold(tmp, null, null, null);
ah2.foldRule(tmp, null, null);
tmp = new ApproximateHistogram(resolution);
}

View File

@ -111,8 +111,8 @@ public class ApproximateHistogramTest
h2.offer(VALUES[i]);
}
merged.fold(h1);
merged.fold(h2);
merged.fold(h1, null, null, null);
merged.fold(h2, null, null, null);
mergedFast.foldFast(h1);
mergedFast.foldFast(h2);
@ -144,7 +144,7 @@ public class ApproximateHistogramTest
ApproximateHistogram aFast = buildHistogram(10, new float[]{1, 2, 3, 4, 5, 6});
ApproximateHistogram b = buildHistogram(5, new float[]{3, 4, 5, 6});
a.fold(b);
a.fold(b, null, null, null);
aFast.foldFast(b);
Assert.assertEquals(
@ -172,7 +172,7 @@ public class ApproximateHistogramTest
for (float v : VALUES4) {
h4.offer(v);
}
h3.fold(h4);
h3.fold(h4, null, null, null);
Assert.assertArrayEquals(
"final bin positions match expected positions",
new float[]{-50.98f, -21.77f, -9.81f, 3.73f, 13.72f, 20.1f, 29f, 44.79f, 53.8f, 64.67f},
@ -191,7 +191,7 @@ public class ApproximateHistogramTest
ApproximateHistogram h1 = new ApproximateHistogram(10);
ApproximateHistogram h2 = new ApproximateHistogram(10);
h1.fold(h2);
h1.fold(h2, null, null, null);
h1.foldFast(h2);
}
@ -210,8 +210,8 @@ public class ApproximateHistogramTest
h4Fast.offer(v);
}
h1.fold(h3);
h4.fold(h2);
h1.fold(h3, null, null, null);
h4.fold(h2, null, null, null);
h1Fast.foldFast(h3);
h4Fast.foldFast(h2);
@ -300,7 +300,7 @@ public class ApproximateHistogramTest
ApproximateHistogram h = buildHistogram(5, VALUES);
Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes()));
ApproximateHistogram h2 = new ApproximateHistogram(50).fold(h);
ApproximateHistogram h2 = new ApproximateHistogram(50).fold(h, null, null, null);
Assert.assertEquals(h2, ApproximateHistogram.fromBytes(h2.toBytes()));
}
@ -331,7 +331,7 @@ public class ApproximateHistogramTest
h = buildHistogram(5, new float[]{1f, 2f, 3f});
Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes()));
h = new ApproximateHistogram(40).fold(h);
h = new ApproximateHistogram(40).fold(h, null, null, null);
Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes()));
}

View File

@ -49,7 +49,7 @@ import java.util.Comparator;
*/
public class VarianceAggregatorCollector
{
public static boolean isVariancePop(String estimator)
public static boolean isVariancePop(@Nullable String estimator)
{
return estimator != null && "population".equalsIgnoreCase(estimator);
}

View File

@ -53,6 +53,7 @@ public class VarianceAggregatorFactory extends AggregatorFactory
{
protected final String fieldName;
protected final String name;
@Nullable
protected final String estimator;
private final String inputType;
@ -62,8 +63,8 @@ public class VarianceAggregatorFactory extends AggregatorFactory
public VarianceAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName,
@JsonProperty("estimator") String estimator,
@JsonProperty("inputType") String inputType
@JsonProperty("estimator") @Nullable String estimator,
@JsonProperty("inputType") @Nullable String inputType
)
{
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
@ -238,6 +239,7 @@ public class VarianceAggregatorFactory extends AggregatorFactory
return name;
}
@Nullable
@JsonProperty
public String getEstimator()
{

View File

@ -22,6 +22,8 @@ package org.apache.druid.query.aggregation.variance;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import javax.annotation.Nullable;
/**
*/
@JsonTypeName("varianceFold")
@ -30,7 +32,7 @@ public class VarianceFoldingAggregatorFactory extends VarianceAggregatorFactory
public VarianceFoldingAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName,
@JsonProperty("estimator") String estimator
@JsonProperty("estimator") @Nullable String estimator
)
{
super(name, fieldName, estimator, "variance");

View File

@ -40,7 +40,7 @@ public class DoubleMaxAggregatorFactory extends SimpleDoubleAggregatorFactory
public DoubleMaxAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") final String fieldName,
@JsonProperty("expression") String expression,
@JsonProperty("expression") @Nullable String expression,
@JacksonInject ExprMacroTable macroTable
)
{

View File

@ -40,7 +40,7 @@ public class DoubleMinAggregatorFactory extends SimpleDoubleAggregatorFactory
public DoubleMinAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") final String fieldName,
@JsonProperty("expression") String expression,
@JsonProperty("expression") @Nullable String expression,
@JacksonInject ExprMacroTable macroTable
)
{

View File

@ -42,7 +42,7 @@ public class DoubleSumAggregatorFactory extends SimpleDoubleAggregatorFactory
public DoubleSumAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") final String fieldName,
@JsonProperty("expression") String expression,
@JsonProperty("expression") @Nullable String expression,
@JacksonInject ExprMacroTable macroTable
)
{

View File

@ -40,7 +40,7 @@ public class FloatMaxAggregatorFactory extends SimpleFloatAggregatorFactory
public FloatMaxAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") final String fieldName,
@JsonProperty("expression") String expression,
@JsonProperty("expression") @Nullable String expression,
@JacksonInject ExprMacroTable macroTable
)
{

View File

@ -40,7 +40,7 @@ public class FloatMinAggregatorFactory extends SimpleFloatAggregatorFactory
public FloatMinAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") final String fieldName,
@JsonProperty("expression") String expression,
@JsonProperty("expression") @Nullable String expression,
@JacksonInject ExprMacroTable macroTable
)
{

View File

@ -42,7 +42,7 @@ public class FloatSumAggregatorFactory extends SimpleFloatAggregatorFactory
public FloatSumAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") final String fieldName,
@JsonProperty("expression") String expression,
@JsonProperty("expression") @Nullable String expression,
@JacksonInject ExprMacroTable macroTable
)
{

View File

@ -40,7 +40,7 @@ public class LongMaxAggregatorFactory extends SimpleLongAggregatorFactory
public LongMaxAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") final String fieldName,
@JsonProperty("expression") String expression,
@JsonProperty("expression") @Nullable String expression,
@JacksonInject ExprMacroTable macroTable
)
{

View File

@ -40,7 +40,7 @@ public class LongMinAggregatorFactory extends SimpleLongAggregatorFactory
public LongMinAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") final String fieldName,
@JsonProperty("expression") String expression,
@JsonProperty("expression") @Nullable String expression,
@JacksonInject ExprMacroTable macroTable
)
{

View File

@ -42,7 +42,7 @@ public class LongSumAggregatorFactory extends SimpleLongAggregatorFactory
public LongSumAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") final String fieldName,
@JsonProperty("expression") String expression,
@JsonProperty("expression") @Nullable String expression,
@JacksonInject ExprMacroTable macroTable
)
{

View File

@ -40,6 +40,7 @@ public interface PostAggregator extends Cacheable
@Nullable
Object compute(Map<String, Object> combinedAggregators);
@Nullable
String getName();
/**

View File

@ -23,10 +23,12 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.collections.SerializablePair;
import javax.annotation.Nullable;
public class SerializablePairLongString extends SerializablePair<Long, String>
{
@JsonCreator
public SerializablePairLongString(@JsonProperty("lhs") Long lhs, @JsonProperty("rhs") String rhs)
public SerializablePairLongString(@JsonProperty("lhs") Long lhs, @JsonProperty("rhs") @Nullable String rhs)
{
super(lhs, rhs);
}

View File

@ -108,7 +108,7 @@ public class StringLastBufferAggregator implements BufferAggregator
mutationBuffer.position(position);
Long timeValue = mutationBuffer.getLong(position);
Integer stringSizeBytes = mutationBuffer.getInt(position + Long.BYTES);
int stringSizeBytes = mutationBuffer.getInt(position + Long.BYTES);
SerializablePairLongString serializablePair;

View File

@ -29,6 +29,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
import javax.annotation.Nullable;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
@ -71,7 +73,7 @@ public class ArithmeticPostAggregator implements PostAggregator
@JsonProperty("name") String name,
@JsonProperty("fn") String fnName,
@JsonProperty("fields") List<PostAggregator> fields,
@JsonProperty("ordering") String ordering
@JsonProperty("ordering") @Nullable String ordering
)
{
Preconditions.checkArgument(fnName != null, "fn cannot not be null");

View File

@ -74,7 +74,7 @@ public class ExpressionPostAggregator implements PostAggregator
public ExpressionPostAggregator(
@JsonProperty("name") String name,
@JsonProperty("expression") String expression,
@JsonProperty("ordering") String ordering,
@JsonProperty("ordering") @Nullable String ordering,
@JacksonInject ExprMacroTable macroTable
)
{

View File

@ -27,6 +27,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
import javax.annotation.Nullable;
import java.util.Comparator;
import java.util.Map;
import java.util.Set;
@ -35,12 +37,13 @@ import java.util.Set;
*/
public class FieldAccessPostAggregator implements PostAggregator
{
@Nullable
private final String name;
private final String fieldName;
@JsonCreator
public FieldAccessPostAggregator(
@JsonProperty("name") String name,
@JsonProperty("name") @Nullable String name,
@JsonProperty("fieldName") String fieldName
)
{
@ -67,6 +70,7 @@ public class FieldAccessPostAggregator implements PostAggregator
return combinedAggregators.get(fieldName);
}
@Nullable
@Override
@JsonProperty
public String getName()

View File

@ -802,7 +802,7 @@ public class GroupByQuery extends BaseQuery<ResultRow>
return new Builder(this).setQuerySegmentSpec(spec).build();
}
public GroupByQuery withDimFilter(final DimFilter dimFilter)
public GroupByQuery withDimFilter(@Nullable final DimFilter dimFilter)
{
return new Builder(this).setDimFilter(dimFilter).build();
}
@ -828,7 +828,7 @@ public class GroupByQuery extends BaseQuery<ResultRow>
return new Builder(this).setAggregatorSpecs(aggregatorSpecs).build();
}
public GroupByQuery withSubtotalsSpec(final List<List<String>> subtotalsSpec)
public GroupByQuery withSubtotalsSpec(@Nullable final List<List<String>> subtotalsSpec)
{
return new Builder(this).setSubtotalsSpec(subtotalsSpec).build();
}
@ -1006,7 +1006,7 @@ public class GroupByQuery extends BaseQuery<ResultRow>
return this;
}
public Builder setSubtotalsSpec(List<List<String>> subtotalsSpec)
public Builder setSubtotalsSpec(@Nullable List<List<String>> subtotalsSpec)
{
this.subtotalsSpec = subtotalsSpec;
return this;
@ -1017,7 +1017,7 @@ public class GroupByQuery extends BaseQuery<ResultRow>
return addOrderByColumn(dimension, null);
}
public Builder addOrderByColumn(String dimension, OrderByColumnSpec.Direction direction)
public Builder addOrderByColumn(String dimension, @Nullable OrderByColumnSpec.Direction direction)
{
return addOrderByColumn(new OrderByColumnSpec(dimension, direction));
}
@ -1059,7 +1059,7 @@ public class GroupByQuery extends BaseQuery<ResultRow>
return this;
}
public Builder setDimFilter(DimFilter dimFilter)
public Builder setDimFilter(@Nullable DimFilter dimFilter)
{
this.dimFilter = dimFilter;
return this;

View File

@ -22,6 +22,8 @@ package org.apache.druid.query.groupby.epinephelinae;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
public class ByteBufferHashTable
@ -74,7 +76,7 @@ public class ByteBufferHashTable
// how many times the table buffer has filled/readjusted (through adjustTableWhenFull())
protected int growthCount;
@Nullable
protected BucketUpdateHandler bucketUpdateHandler;
public ByteBufferHashTable(
@ -84,7 +86,7 @@ public class ByteBufferHashTable
ByteBuffer buffer,
int keySize,
int maxSizeForTesting,
BucketUpdateHandler bucketUpdateHandler
@Nullable BucketUpdateHandler bucketUpdateHandler
)
{
this.maxLoadFactor = maxLoadFactor;

View File

@ -37,6 +37,7 @@ import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper.RowBas
import org.apache.druid.query.groupby.resource.GroupByQueryResource;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.File;
import java.nio.ByteBuffer;

View File

@ -47,6 +47,8 @@ import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.ObjectColumnSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@ -435,6 +437,7 @@ public class ParallelCombiner<KeyType>
private static final int UNKNOWN_COLUMN_INDEX = -1;
private final Object2IntMap<String> columnIndexMap;
@Nullable
private Object[] values;
SettableColumnSelectorFactory(AggregatorFactory[] aggregatorFactories)
@ -446,7 +449,7 @@ public class ParallelCombiner<KeyType>
}
}
public void set(Object[] values)
public void set(@Nullable Object[] values)
{
this.values = values;
}

View File

@ -67,7 +67,7 @@ public final class DimensionHandlerUtils
public static DimensionHandler getHandlerFromCapabilities(
String dimensionName,
ColumnCapabilities capabilities,
MultiValueHandling multiValueHandling
@Nullable MultiValueHandling multiValueHandling
)
{
if (capabilities == null) {

View File

@ -125,7 +125,7 @@ public interface DimensionIndexer
* @param reportParseExceptions
* @return An array containing an encoded representation of the input row value.
*/
EncodedKeyComponentType processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolean reportParseExceptions);
EncodedKeyComponentType processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions);
/**
* Gives the estimated size in bytes for the given key

View File

@ -39,7 +39,7 @@ public class DoubleDimensionIndexer implements DimensionIndexer<Double, Double,
public static final Comparator<Double> DOUBLE_COMPARATOR = Comparators.naturalNullsFirst();
@Override
public Double processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolean reportParseExceptions)
public Double processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions)
{
if (dimValues instanceof List) {
throw new UnsupportedOperationException("Numeric columns do not support multivalue rows.");

View File

@ -39,7 +39,7 @@ public class FloatDimensionIndexer implements DimensionIndexer<Float, Float, Flo
public static final Comparator<Float> FLOAT_COMPARATOR = Comparators.naturalNullsFirst();
@Override
public Float processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolean reportParseExceptions)
public Float processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions)
{
if (dimValues instanceof List) {
throw new UnsupportedOperationException("Numeric columns do not support multivalue rows.");

View File

@ -29,6 +29,8 @@ import org.apache.druid.segment.data.CompressionFactory;
import org.apache.druid.segment.data.CompressionStrategy;
import org.apache.druid.segment.data.ConciseBitmapSerdeFactory;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.Objects;
import java.util.Set;
@ -90,10 +92,10 @@ public class IndexSpec
*/
@JsonCreator
public IndexSpec(
@JsonProperty("bitmap") BitmapSerdeFactory bitmapSerdeFactory,
@JsonProperty("dimensionCompression") CompressionStrategy dimensionCompression,
@JsonProperty("metricCompression") CompressionStrategy metricCompression,
@JsonProperty("longEncoding") CompressionFactory.LongEncodingStrategy longEncoding
@JsonProperty("bitmap") @Nullable BitmapSerdeFactory bitmapSerdeFactory,
@JsonProperty("dimensionCompression") @Nullable CompressionStrategy dimensionCompression,
@JsonProperty("metricCompression") @Nullable CompressionStrategy metricCompression,
@JsonProperty("longEncoding") @Nullable CompressionFactory.LongEncodingStrategy longEncoding
)
{
Preconditions.checkArgument(dimensionCompression == null || DIMENSION_COMPRESSION.contains(dimensionCompression),

View File

@ -39,7 +39,7 @@ public class LongDimensionIndexer implements DimensionIndexer<Long, Long, Long>
public static final Comparator LONG_COMPARATOR = Comparators.<Long>naturalNullsFirst();
@Override
public Long processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolean reportParseExceptions)
public Long processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions)
{
if (dimValues instanceof List) {
throw new UnsupportedOperationException("Numeric columns do not support multivalue rows.");

View File

@ -87,7 +87,7 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
valueToId.defaultReturnValue(-1);
}
public int getId(String value)
public int getId(@Nullable String value)
{
lock.readLock().lock();
try {
@ -245,7 +245,7 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
}
@Override
public int[] processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolean reportParseExceptions)
public int[] processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions)
{
final int[] encodedDimensionValues;
final int oldDictSize = dimLookup.size();

View File

@ -284,7 +284,7 @@ public class GenericIndexedWriter<T> implements Serializer
}
@Override
public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
public void writeTo(WritableByteChannel channel, @Nullable FileSmoosher smoosher) throws IOException
{
if (requireMultipleFiles) {
writeToMultiFiles(channel, smoosher);

View File

@ -996,7 +996,7 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
return iterableWithPostAggregations(null, false).iterator();
}
public Iterable<Row> iterableWithPostAggregations(final List<PostAggregator> postAggs, final boolean descending)
public Iterable<Row> iterableWithPostAggregations(@Nullable final List<PostAggregator> postAggs, final boolean descending)
{
return () -> {
final List<DimensionDesc> dimensions = getDimensions();

View File

@ -23,6 +23,8 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import javax.annotation.Nullable;
import java.util.List;
/**
@ -45,18 +47,21 @@ public final class ChangeRequestsSnapshot<T>
private final boolean resetCounter;
/** cause for reset if {@link #resetCounter} is true */
@Nullable
private final String resetCause;
/** segments requests delta since counter, if {@link #resetCounter} if false */
@Nullable
private final ChangeRequestHistory.Counter counter;
@Nullable
private final List<T> requests;
@JsonCreator
public ChangeRequestsSnapshot(
@JsonProperty("resetCounter") boolean resetCounter,
@JsonProperty("resetCause") String resetCause,
@JsonProperty("counter") ChangeRequestHistory.Counter counter,
@JsonProperty("requests") List<T> requests
@JsonProperty("resetCause") @Nullable String resetCause,
@JsonProperty("counter") @Nullable ChangeRequestHistory.Counter counter,
@JsonProperty("requests") @Nullable List<T> requests
)
{
this.resetCounter = resetCounter;
@ -76,18 +81,21 @@ public final class ChangeRequestsSnapshot<T>
return resetCounter;
}
@Nullable
@JsonProperty
public String getResetCause()
{
return resetCause;
}
@Nullable
@JsonProperty
public ChangeRequestHistory.Counter getCounter()
{
return counter;
}
@Nullable
@JsonProperty
public List<T> getRequests()
{

View File

@ -21,10 +21,12 @@ package org.apache.druid.server.coordination;
import org.apache.druid.timeline.DataSegment;
import javax.annotation.Nullable;
/**
*/
public interface DataSegmentChangeHandler
{
void addSegment(DataSegment segment, DataSegmentChangeCallback callback);
void removeSegment(DataSegment segment, DataSegmentChangeCallback callback);
void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback);
void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback);
}

View File

@ -22,6 +22,8 @@ package org.apache.druid.server.coordination;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import javax.annotation.Nullable;
/**
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "action")
@ -32,7 +34,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
})
public interface DataSegmentChangeRequest
{
void go(DataSegmentChangeHandler handler, DataSegmentChangeCallback callback);
void go(DataSegmentChangeHandler handler, @Nullable DataSegmentChangeCallback callback);
String asString();
}

View File

@ -25,6 +25,8 @@ import com.fasterxml.jackson.annotation.JsonUnwrapped;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import java.util.Objects;
/**
@ -49,7 +51,7 @@ public class SegmentChangeRequestDrop implements DataSegmentChangeRequest
}
@Override
public void go(DataSegmentChangeHandler handler, DataSegmentChangeCallback callback)
public void go(DataSegmentChangeHandler handler, @Nullable DataSegmentChangeCallback callback)
{
handler.removeSegment(segment, callback);
}

View File

@ -25,6 +25,8 @@ import com.fasterxml.jackson.annotation.JsonUnwrapped;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import java.util.Objects;
/**
@ -42,7 +44,7 @@ public class SegmentChangeRequestLoad implements DataSegmentChangeRequest
}
@Override
public void go(DataSegmentChangeHandler handler, DataSegmentChangeCallback callback)
public void go(DataSegmentChangeHandler handler, @Nullable DataSegmentChangeCallback callback)
{
handler.addSegment(segment, callback);
}

View File

@ -19,15 +19,19 @@
package org.apache.druid.server.coordination;
import javax.annotation.Nullable;
/**
*/
public class SegmentChangeRequestNoop implements DataSegmentChangeRequest
{
@Override
public void go(DataSegmentChangeHandler handler, DataSegmentChangeCallback callback)
public void go(DataSegmentChangeHandler handler, @Nullable DataSegmentChangeCallback callback)
{
// just execute the callback and do nothing else
callback.execute();
if (callback != null) {
callback.execute();
}
}
@Override