Make @Override annotation obligatory (#4274)

* Make MissingOverride an error

* Make travis stript to fail fast

* Add missing Override annotations

* Comment
This commit is contained in:
Roman Leventov 2017-05-16 13:30:30 -05:00 committed by Himanshu
parent 8333043b7b
commit b7a52286e8
134 changed files with 362 additions and 18 deletions

View File

@ -3,9 +3,8 @@ language: java
jdk:
- oraclejdk8
script:
- mvn clean -Pstrict -pl '!benchmarks' compile test-compile -B
- mvn test -B -Pparallel-test -Dmaven.fork.count=2
# Using && instead of two script steps to make Travis build fail faster, if "strict" compilation is not successful
script: mvn clean -Pstrict -pl '!benchmarks' compile test-compile -B && mvn test -B -Pparallel-test -Dmaven.fork.count=2
install: true

View File

@ -76,6 +76,7 @@ public class MapBasedRow implements Row
return timestamp.getMillis();
}
@Override
@JsonProperty
public DateTime getTimestamp()
{

View File

@ -42,6 +42,7 @@ public class TimeAndDimsParseSpec extends ParseSpec
);
}
@Override
public Parser<String, Object> makeParser()
{
return new Parser<String, Object>()
@ -66,11 +67,13 @@ public class TimeAndDimsParseSpec extends ParseSpec
};
}
@Override
public ParseSpec withTimestampSpec(TimestampSpec spec)
{
return new TimeAndDimsParseSpec(spec, getDimensionsSpec());
}
@Override
public ParseSpec withDimensionsSpec(DimensionsSpec spec)
{
return new TimeAndDimsParseSpec(getTimestampSpec(), spec);

View File

@ -25,6 +25,7 @@ public class Runnables
{
public static Runnable getNoopRunnable(){
return new Runnable(){
@Override
public void run(){}
};
}

View File

@ -78,6 +78,7 @@ public class RectangularBound implements Bound
return maxCoords;
}
@Override
@JsonProperty
public int getLimit()
{

View File

@ -229,4 +229,24 @@
</plugins>
</build>
<profiles>
<profile>
<id>strict</id>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<compilerArgs>
<!-- Antlr-generated classes miss @Override, that is not easy to fix -->
<arg>-Xep:MissingOverride:WARN</arg>
</compilerArgs>
</configuration>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View File

@ -183,16 +183,19 @@ public class CombiningSequence<T> implements Sequence<T>
return accumulator;
}
@Override
public void reset()
{
accumulator.reset();
}
@Override
public boolean yielded()
{
return accumulator.yielded();
}
@Override
public void yield()
{
accumulator.yield();

View File

@ -159,6 +159,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
}
}
@Override
public PartitionHolder<ObjectType> findEntry(Interval interval, VersionType version)
{
try {
@ -190,6 +191,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
* @return Holders representing the interval that the objects exist for, PartitionHolders
* are guaranteed to be complete
*/
@Override
public List<TimelineObjectHolder<VersionType, ObjectType>> lookup(Interval interval)
{
try {

View File

@ -71,6 +71,7 @@ public class ExecsTest
producer.submit(
new Runnable()
{
@Override
public void run()
{
for (int i = 0; i < nTasks; i++) {

View File

@ -193,16 +193,19 @@ public class LifecycleScopeTest
++ran;
}
@Override
public int getStarted()
{
return started;
}
@Override
public int getStopped()
{
return stopped;
}
@Override
public int getRan()
{
return ran;
@ -233,16 +236,19 @@ public class LifecycleScopeTest
++ran;
}
@Override
public int getStarted()
{
return started;
}
@Override
public int getStopped()
{
return stopped;
}
@Override
public int getRan()
{
return ran;

View File

@ -213,6 +213,7 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory<InputRowP
private final Runnable doNothingRunnable = new Runnable()
{
@Override
public void run()
{
}

View File

@ -605,6 +605,7 @@ public interface IntSet extends Cloneable, Comparable<IntSet>
/**
* @return <tt>true</tt> if the iterator has more elements.
*/
@Override
boolean hasNext();
/**
@ -612,6 +613,7 @@ public interface IntSet extends Cloneable, Comparable<IntSet>
*
* @throws NoSuchElementException iteration has no more elements.
*/
@Override
int next();
/**
@ -648,6 +650,7 @@ public interface IntSet extends Cloneable, Comparable<IntSet>
*
* @return a clone of the IntIterator
*/
@Override
public IntIterator clone();
}
}

View File

@ -73,6 +73,7 @@ public class DruidParquetReadSupport extends AvroReadSupport<GenericRecord>
return new MessageType(name, partialFields);
}
@Override
public ReadContext init(InitContext context)
{
MessageType requestedProjection = getSchemaForRead(context.getFileSchema(), getPartialReadSchema(context));

View File

@ -30,6 +30,7 @@ import java.util.Arrays;
import java.util.List;
public class ScanQueryDruidModule implements DruidModule {
@Override
public void configure(Binder binder) {
DruidBinders.queryToolChestBinder(binder)
.addBinding(ScanQuery.class)
@ -42,6 +43,7 @@ public class ScanQueryDruidModule implements DruidModule {
.in(LazySingleton.class);
}
@Override
public List<? extends Module> getJacksonModules() {
return Arrays.<Module>asList(
new SimpleModule("ScanQueryDruidModule")

View File

@ -96,4 +96,24 @@
</plugin>
</plugins>
</build>
<profiles>
<profile>
<id>strict</id>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<compilerArgs>
<!-- Thrift-generated classes miss @Override, that is not easy to fix -->
<arg>-Xep:MissingOverride:WARN</arg>
</compilerArgs>
</configuration>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View File

@ -226,6 +226,7 @@ public class MapVirtualColumn implements VirtualColumn
return valueDimension;
}
@Override
@JsonProperty
public String getOutputName()
{

View File

@ -140,4 +140,24 @@
</plugin>
</plugins>
</build>
<profiles>
<profile>
<id>strict</id>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<compilerArgs>
<!-- Avro-generated classes miss @Override, that is not easy to fix -->
<arg>-Xep:MissingOverride:WARN</arg>
</compilerArgs>
</configuration>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View File

@ -109,6 +109,7 @@ public class AvroStreamInputRowParserTest
public static final ByteBuffer SOME_BYTES_VALUE = ByteBuffer.allocate(8);
private static final Function<Object, String> TO_STRING_INCLUDING_NULL = new Function<Object, String>()
{
@Override
public String apply(Object o) { return String.valueOf(o); }
};

View File

@ -107,6 +107,7 @@ public class ApproximateHistogramFoldingSerde extends ComplexMetricSerde
return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy());
}
@Override
public ObjectStrategy getObjectStrategy()
{
return new ObjectStrategy<ApproximateHistogram>()

View File

@ -98,6 +98,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
);
}
@Override
@JsonProperty
public int getMaxRowsInMemory()
{
@ -110,24 +111,28 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
return maxRowsPerSegment;
}
@Override
@JsonProperty
public Period getIntermediatePersistPeriod()
{
return intermediatePersistPeriod;
}
@Override
@JsonProperty
public File getBasePersistDirectory()
{
return basePersistDirectory;
}
@Override
@JsonProperty
public int getMaxPendingPersists()
{
return maxPendingPersists;
}
@Override
@JsonProperty
public IndexSpec getIndexSpec()
{
@ -140,6 +145,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
return buildV9Directly;
}
@Override
@JsonProperty
public boolean isReportParseExceptions()
{

View File

@ -44,6 +44,7 @@ public final class CacheHandler implements AutoCloseable
return cache;
}
@Override
public void close()
{
cacheManager.disposeCache(this);

View File

@ -128,5 +128,6 @@ public interface LoadingCache<K, V> extends Closeable
* the namespace deletion event should call this method to clean up resources.
*/
@Override
void close();
}

View File

@ -54,6 +54,7 @@ public class AWSSessionCredentialsAdapter extends AWSSessionCredentials {
return provider.getCredentials().getAWSSecretKey();
}
@Override
public String getSessionToken() {
com.amazonaws.auth.AWSSessionCredentials sessionCredentials =
(com.amazonaws.auth.AWSSessionCredentials) provider.getCredentials();

View File

@ -363,6 +363,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller, URIDataPuller
this.path = key;
}
@Override
public String toString()
{
return String.format("s3://%s/%s", bucket, path);

View File

@ -108,6 +108,7 @@ public class S3TaskLogs implements TaskLogs
}
}
@Override
public void pushTaskLog(final String taskid, final File logFile) throws IOException
{
final String taskKey = getTaskLogKey(taskid);

View File

@ -75,6 +75,7 @@ public class DetermineHashedPartitionsJob implements Jobby
this.config = config;
}
@Override
public boolean run()
{
try {

View File

@ -104,6 +104,7 @@ public class DeterminePartitionsJob implements Jobby
this.config = config;
}
@Override
public boolean run()
{
try {

View File

@ -153,6 +153,7 @@ public class IndexGeneratorJob implements Jobby
return jobStats;
}
@Override
public boolean run()
{
try {

View File

@ -50,6 +50,7 @@ public class FSSpideringIterator implements Iterator<FileStatus>
{
return new Iterable<FileStatus>()
{
@Override
public Iterator<FileStatus> iterator()
{
return spiderPathPropagateExceptions(fs, path);
@ -72,6 +73,7 @@ public class FSSpideringIterator implements Iterator<FileStatus>
this.statii = statii;
}
@Override
public boolean hasNext()
{
if (statuses != null && !statuses.hasNext()) {
@ -81,6 +83,7 @@ public class FSSpideringIterator implements Iterator<FileStatus>
return index < statii.length;
}
@Override
public FileStatus next()
{
while (hasNext()) {
@ -98,6 +101,7 @@ public class FSSpideringIterator implements Iterator<FileStatus>
throw new NoSuchElementException();
}
@Override
public void remove()
{
throw new UnsupportedOperationException();

View File

@ -52,18 +52,21 @@ public abstract class AbstractPartitionsSpec implements PartitionsSpec
);
}
@Override
@JsonProperty
public long getTargetPartitionSize()
{
return targetPartitionSize;
}
@Override
@JsonProperty
public long getMaxPartitionSize()
{
return maxPartitionSize;
}
@Override
@JsonProperty
public boolean isAssumeGrouped()
{

View File

@ -47,6 +47,7 @@ public class LockAcquireAction implements TaskAction<TaskLock>
return interval;
}
@Override
public TypeReference<TaskLock> getReturnTypeReference()
{
return new TypeReference<TaskLock>()

View File

@ -27,6 +27,7 @@ import java.util.List;
public class LockListAction implements TaskAction<List<TaskLock>>
{
@Override
public TypeReference<List<TaskLock>> getReturnTypeReference()
{
return new TypeReference<List<TaskLock>>() {};

View File

@ -45,6 +45,7 @@ public class LockReleaseAction implements TaskAction<Void>
return interval;
}
@Override
public TypeReference<Void> getReturnTypeReference()
{
return new TypeReference<Void>() {};

View File

@ -46,6 +46,7 @@ public class LockTryAcquireAction implements TaskAction<TaskLock>
return interval;
}
@Override
public TypeReference<TaskLock> getReturnTypeReference()
{
return new TypeReference<TaskLock>()

View File

@ -56,6 +56,7 @@ public class SegmentInsertAction implements TaskAction<Set<DataSegment>>
return segments;
}
@Override
public TypeReference<Set<DataSegment>> getReturnTypeReference()
{
return new TypeReference<Set<DataSegment>>()

View File

@ -60,6 +60,7 @@ public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
return interval;
}
@Override
public TypeReference<List<DataSegment>> getReturnTypeReference()
{
return new TypeReference<List<DataSegment>>() {};

View File

@ -76,6 +76,7 @@ public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
return intervals;
}
@Override
public TypeReference<List<DataSegment>> getReturnTypeReference()
{
return new TypeReference<List<DataSegment>>() {};

View File

@ -51,6 +51,7 @@ public class SegmentMetadataUpdateAction implements TaskAction<Void>
return segments;
}
@Override
public TypeReference<Void> getReturnTypeReference()
{
return new TypeReference<Void>() {};

View File

@ -51,6 +51,7 @@ public class SegmentNukeAction implements TaskAction<Void>
return segments;
}
@Override
public TypeReference<Void> getReturnTypeReference()
{
return new TypeReference<Void>()

View File

@ -84,6 +84,7 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
return endMetadata;
}
@Override
public TypeReference<SegmentPublishResult> getReturnTypeReference()
{
return new TypeReference<SegmentPublishResult>()

View File

@ -147,6 +147,7 @@ public class HadoopIndexTask extends HadoopTask
return spec;
}
@Override
@JsonProperty
public List<String> getHadoopDependencyCoordinates()
{

View File

@ -516,6 +516,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
}
@Override
@LifecycleStop
public void stop()
{

View File

@ -428,12 +428,14 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
return ImmutableList.copyOf(pendingTasks.values());
}
@Override
public Collection<Task> getPendingTaskPayloads()
{
// return a snapshot of current pending task payloads.
return ImmutableList.copyOf(pendingTaskPayloads.values());
}
@Override
public RemoteTaskRunnerConfig getConfig()
{
return config;

View File

@ -41,30 +41,35 @@ public class PendingTaskBasedWorkerResourceManagementConfig extends SimpleWorker
return this;
}
@Override
public PendingTaskBasedWorkerResourceManagementConfig setWorkerIdleTimeout(Period workerIdleTimeout)
{
super.setWorkerIdleTimeout(workerIdleTimeout);
return this;
}
@Override
public PendingTaskBasedWorkerResourceManagementConfig setMaxScalingDuration(Period maxScalingDuration)
{
super.setMaxScalingDuration(maxScalingDuration);
return this;
}
@Override
public PendingTaskBasedWorkerResourceManagementConfig setNumEventsToTrack(int numEventsToTrack)
{
super.setNumEventsToTrack(numEventsToTrack);
return this;
}
@Override
public PendingTaskBasedWorkerResourceManagementConfig setWorkerVersion(String workerVersion)
{
super.setWorkerVersion(workerVersion);
return this;
}
@Override
public PendingTaskBasedWorkerResourceManagementConfig setPendingTaskTimeout(Period pendingTaskTimeout)
{
super.setPendingTaskTimeout(pendingTaskTimeout);

View File

@ -94,6 +94,7 @@ public class SimpleWorkerResourceManagementStrategy extends AbstractWorkerResour
}
@Override
protected boolean doProvision(WorkerTaskRunner runner)
{
Collection<? extends TaskRunnerWorkItem> pendingTasks = runner.getPendingTasks();
@ -163,6 +164,7 @@ public class SimpleWorkerResourceManagementStrategy extends AbstractWorkerResour
}
}
@Override
boolean doTerminate(WorkerTaskRunner runner)
{
Collection<? extends TaskRunnerWorkItem> pendingTasks = runner.getPendingTasks();

View File

@ -65,6 +65,7 @@ public class RemoteTaskRunnerRunPendingTasksConcurrencyTest
remoteTaskRunner = rtrTestUtils.makeRemoteTaskRunner(
new TestRemoteTaskRunnerConfig(new Period("PT3600S"))
{
@Override
public int getPendingTasksRunnerNumThreads()
{
return 2;

View File

@ -41,6 +41,7 @@ public class CountingMap<K> extends AbstractMap<K, Long>
counts.get(k).addAndGet(n);
}
@Override
public Set<Entry<K, Long>> entrySet()
{
return Maps.transformValues(

View File

@ -192,13 +192,7 @@ public class ScheduledExecutors
public static ScheduledExecutorFactory createFactory(final Lifecycle lifecycle)
{
return new ScheduledExecutorFactory()
{
public ScheduledExecutorService create(int corePoolSize, String nameFormat)
{
return ExecutorServices.manageLifecycle(lifecycle, fixed(corePoolSize, nameFormat));
}
};
return (corePoolSize, nameFormat) -> ExecutorServices.manageLifecycle(lifecycle, fixed(corePoolSize, nameFormat));
}
public static ScheduledExecutorService fixed(int corePoolSize, String nameFormat)

View File

@ -37,6 +37,7 @@ public class DroppingIterable<T> implements Iterable<T>
this.numToDrop = numToDrop;
}
@Override
public Iterator<T> iterator()
{
return new DroppingIterator<>(delegate.iterator(), numToDrop);

View File

@ -38,6 +38,7 @@ public class DroppingIterator<T> implements Iterator<T>
this.numToDrop = numToDrop;
}
@Override
public boolean hasNext()
{
if (! dropped) {
@ -50,6 +51,7 @@ public class DroppingIterator<T> implements Iterator<T>
return delegate.hasNext();
}
@Override
public T next()
{
if (! dropped) {
@ -61,6 +63,7 @@ public class DroppingIterator<T> implements Iterator<T>
return delegate.next();
}
@Override
public void remove()
{
throw new UnsupportedOperationException();

View File

@ -58,6 +58,7 @@ public class FunctionalIterable<T> implements Iterable<T>
this.delegate = delegate;
}
@Override
public Iterator<T> iterator()
{
return delegate.iterator();

View File

@ -58,16 +58,19 @@ public class FunctionalIterator<T> implements Iterator<T>
this.delegate = delegate;
}
@Override
public boolean hasNext()
{
return delegate.hasNext();
}
@Override
public T next()
{
return delegate.next();
}
@Override
public void remove()
{
delegate.remove();

View File

@ -44,6 +44,7 @@ public class JavaScriptParser implements Parser<String, Object>
return new Function<Object, Object>()
{
@Override
public Object apply(Object input)
{
// ideally we need a close() function to discard the context once it is not used anymore

View File

@ -1022,8 +1022,11 @@
<showWarnings>false</showWarnings>
<compilerArgs>
<arg>-XepDisableWarningsInGeneratedCode</arg>
<arg>-Xep:ClassCanBeStatic:ERROR</arg>
<arg>-Xep:PreconditionsInvalidPlaceholder:ERROR</arg>
<arg>-Xep:MissingOverride:ERROR</arg>
<arg>-Xep:ArgumentParameterSwap</arg>
<arg>-Xep:AssistedInjectAndInjectOnSameConstructor</arg>
<arg>-Xep:AutoFactoryAtInject</arg>

View File

@ -52,6 +52,7 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig implem
return Math.max(2, getNumThreads() / 4);
}
@Override
@Config(value = "${base_path}.columnCache.sizeBytes")
public int columnCacheSizeBytes()
{

View File

@ -49,6 +49,7 @@ public class QueryDataSource implements DataSource
return query;
}
@Override
public String toString() { return query.toString(); }
@Override

View File

@ -48,6 +48,7 @@ public class TableDataSource implements DataSource
return Collections.singletonList(name);
}
@Override
public String toString() { return name; }
@Override

View File

@ -41,6 +41,7 @@ public interface Aggregator extends Closeable
Object get();
float getFloat();
@Override
void close();
long getLong();

View File

@ -123,6 +123,7 @@ public interface BufferAggregator extends HotLoopCallee
* <p>The default implementation inspects nothing. Classes that implement {@code BufferAggregator} are encouraged to
* override this method, following the specification of {@link HotLoopCallee#inspectRuntimeShape}.
*/
@Override
default void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
}

View File

@ -197,6 +197,7 @@ public class ArithmeticPostAggregator implements PostAggregator
{
PLUS("+")
{
@Override
public double compute(double lhs, double rhs)
{
return lhs + rhs;
@ -204,6 +205,7 @@ public class ArithmeticPostAggregator implements PostAggregator
},
MINUS("-")
{
@Override
public double compute(double lhs, double rhs)
{
return lhs - rhs;
@ -211,6 +213,7 @@ public class ArithmeticPostAggregator implements PostAggregator
},
MULT("*")
{
@Override
public double compute(double lhs, double rhs)
{
return lhs * rhs;
@ -218,6 +221,7 @@ public class ArithmeticPostAggregator implements PostAggregator
},
DIV("/")
{
@Override
public double compute(double lhs, double rhs)
{
return (rhs == 0.0) ? 0 : (lhs / rhs);
@ -225,6 +229,7 @@ public class ArithmeticPostAggregator implements PostAggregator
},
QUOTIENT("quotient")
{
@Override
public double compute(double lhs, double rhs)
{
return lhs / rhs;
@ -267,6 +272,7 @@ public class ArithmeticPostAggregator implements PostAggregator
public static enum Ordering implements Comparator<Double> {
// ensures the following order: numeric > NaN > Infinite
numericFirst {
@Override
public int compare(Double lhs, Double rhs) {
if(isFinite(lhs) && !isFinite(rhs)) {
return 1;

View File

@ -148,6 +148,7 @@ public class ExpressionPostAggregator implements PostAggregator
{
// ensures the following order: numeric > NaN > Infinite
numericFirst {
@Override
public int compare(Number lhs, Number rhs)
{
if (lhs instanceof Long && rhs instanceof Long) {

View File

@ -68,6 +68,7 @@ public class JavaScriptPostAggregator implements PostAggregator
return new Function()
{
@Override
public double apply(Object[] args)
{
// ideally we need a close() function to discard the context once it is not used anymore

View File

@ -34,31 +34,37 @@ public class CascadeExtractionFn implements ExtractionFn
private final ChainedExtractionFn DEFAULT_CHAINED_EXTRACTION_FN = new ChainedExtractionFn(
new ExtractionFn()
{
@Override
public byte[] getCacheKey()
{
return new byte[0];
}
@Override
public String apply(Object value)
{
return null;
}
@Override
public String apply(String value)
{
return null;
}
@Override
public String apply(long value)
{
return null;
}
@Override
public boolean preservesOrdering()
{
return false;
}
@Override
public ExtractionType getExtractionType()
{
return ExtractionType.MANY_TO_ONE;
@ -219,6 +225,7 @@ public class CascadeExtractionFn implements ExtractionFn
}
}
@Override
public boolean equals(Object o)
{
if (this == o) {
@ -240,6 +247,7 @@ public class CascadeExtractionFn implements ExtractionFn
return true;
}
@Override
public int hashCode()
{
int result = fn.hashCode();
@ -249,6 +257,7 @@ public class CascadeExtractionFn implements ExtractionFn
return result;
}
@Override
public String toString()
{
return (child != null)

View File

@ -50,6 +50,7 @@ public class JavaScriptExtractionFn implements ExtractionFn
return new Function<Object, String>()
{
@Override
public String apply(Object input)
{
// ideally we need a close() function to discard the context once it is not used anymore

View File

@ -37,6 +37,7 @@ public class NotDimFilter implements DimFilter
{
private static final Function<DimFilter, DimFilter> NEGATE =
new Function<DimFilter, DimFilter>() {
@Override
public DimFilter apply(DimFilter filter) { return Druids.newNotDimFilterBuilder().field(filter).build(); }
};

View File

@ -447,6 +447,7 @@ public class GroupByQueryEngine
throw new UnsupportedOperationException();
}
@Override
public void close()
{
// cleanup

View File

@ -69,6 +69,7 @@ public class DimensionSelectorHavingSpec extends BaseHavingSpec
return extractionFn;
}
@Override
public boolean eval(Row row)
{
List<String> dimRowValList = row.getDimension(dimension);
@ -89,6 +90,7 @@ public class DimensionSelectorHavingSpec extends BaseHavingSpec
return false;
}
@Override
public byte[] getCacheKey()
{
byte[] dimBytes = StringUtils.toUtf8(dimension);
@ -143,6 +145,7 @@ public class DimensionSelectorHavingSpec extends BaseHavingSpec
}
@Override
public String toString()
{
StringBuilder sb = new StringBuilder();

View File

@ -98,6 +98,7 @@ public class StringComparators
{
// This code is based on https://github.com/amjjd/java-alphanum, see
// NOTICE file for more information
@Override
public int compare(String str1, String str2)
{
int[] pos =

View File

@ -70,6 +70,7 @@ public class SearchSortSpec
return ordering.getCacheKey();
}
@Override
public String toString()
{
return String.format("%sSort", ordering.toString());

View File

@ -70,16 +70,19 @@ public abstract class PagingOffset
super(offset, threshold);
}
@Override
public final boolean isDescending()
{
return false;
}
@Override
public final int startDelta()
{
return startOffset;
}
@Override
public final int current()
{
return startOffset + counter;
@ -93,16 +96,19 @@ public abstract class PagingOffset
super(offset, threshold);
}
@Override
public final boolean isDescending()
{
return true;
}
@Override
public final int startDelta()
{
return -startOffset - 1;
}
@Override
public final int current()
{
return startOffset - counter;

View File

@ -145,6 +145,7 @@ public class SelectQuery extends BaseQuery<Result<SelectResultValue>>
return pagingSpec.getOffset(identifier, isDescending());
}
@Override
public SelectQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec)
{
return Druids.SelectQueryBuilder.copy(this).intervals(querySegmentSpec).build();
@ -156,6 +157,7 @@ public class SelectQuery extends BaseQuery<Result<SelectResultValue>>
return Druids.SelectQueryBuilder.copy(this).dataSource(dataSource).build();
}
@Override
public SelectQuery withOverriddenContext(Map<String, Object> contextOverrides)
{
Map<String, Object> newContext = computeOverriddenContext(getContext(), contextOverrides);

View File

@ -129,6 +129,7 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
return getContextBoolean("skipEmptyBuckets", false);
}
@Override
public TimeseriesQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec)
{
return Druids.TimeseriesQueryBuilder.copy(this).intervals(querySegmentSpec).build();
@ -140,6 +141,7 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
return Druids.TimeseriesQueryBuilder.copy(this).dataSource(dataSource).build();
}
@Override
public TimeseriesQuery withOverriddenContext(Map<String, Object> contextOverrides)
{
Map<String, Object> newContext = computeOverriddenContext(getContext(), contextOverrides);

View File

@ -168,6 +168,7 @@ public class TopNQuery extends BaseQuery<Result<TopNResultValue>>
topNMetricSpec.initTopNAlgorithmSelector(selector);
}
@Override
public TopNQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec)
{
return new TopNQueryBuilder(this).intervals(querySegmentSpec).build();
@ -199,6 +200,7 @@ public class TopNQuery extends BaseQuery<Result<TopNResultValue>>
return new TopNQueryBuilder(this).threshold(threshold).build();
}
@Override
public TopNQuery withOverriddenContext(Map<String, Object> contextOverrides)
{
return new TopNQueryBuilder(this).context(computeOverriddenContext(getContext(), contextOverrides)).build();

View File

@ -65,11 +65,13 @@ public class CompressedVSizeIndexedSupplier implements WritableSupplier<IndexedM
this.valueSupplier = valueSupplier;
}
@Override
public long getSerializedSize()
{
return 1 + offsetSupplier.getSerializedSize() + valueSupplier.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{version}));

View File

@ -77,6 +77,7 @@ public class FloatMetricColumnSerializer implements MetricColumnSerializer
closeFile(outFile);
}
@Override
public void closeFile(final File outFile) throws IOException
{
outFile.delete();

View File

@ -1075,6 +1075,7 @@ public class IndexMerger
this.lastVal = NOT_INIT;
}
@Override
public int seek(int dictId)
{
if (dimConversions == null) {

View File

@ -81,6 +81,7 @@ public class LongMetricColumnSerializer implements MetricColumnSerializer
closeFile(outFile);
}
@Override
public void closeFile(final File outFile) throws IOException
{
outFile.delete();

View File

@ -43,5 +43,6 @@ public interface QueryableIndex extends ColumnSelector, Closeable
* @throws java.io.IOException if an exception was thrown closing the index
*/
//@Deprecated // This is still required for SimpleQueryableIndex. It should not go away unitl SimpleQueryableIndex is fixed
@Override
public void close() throws IOException;
}

View File

@ -181,6 +181,7 @@ public class ReferenceCountingSegment extends AbstractSegment
}
}
@Override
public <T> T as(Class<T> clazz)
{
return getBaseSegment().as(clazz);

View File

@ -496,6 +496,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
this.lastVal = NOT_INIT;
}
@Override
public int seek(int dictId)
{
if (dimConversions == null) {

View File

@ -226,6 +226,7 @@ public class VirtualColumns implements Cacheable
return new VirtualizedColumnSelectorFactory(baseFactory, this);
}
@Override
public byte[] getCacheKey()
{
// id doesn't matter as there is only one kind of "VirtualColumns", so use 0.

View File

@ -87,6 +87,7 @@ public class BlockLayoutIndexedLongSupplier implements Supplier<IndexedLongs>
return longBuffer.get(longBuffer.position() + bufferIndex);
}
@Override
protected void loadBuffer(int bufferNum)
{
CloseQuietly.close(holder);

View File

@ -94,6 +94,7 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier<IndexedIn
}
}
@Override
public long getSerializedSize()
{
return 1 + // version
@ -103,6 +104,7 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier<IndexedIn
baseIntBuffers.getSerializedSize(); // data
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{VERSION}));

View File

@ -122,6 +122,7 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier<Inde
}
@Override
public long getSerializedSize()
{
return 1 + // version
@ -132,6 +133,7 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier<Inde
baseBuffers.getSerializedSize(); // data
}
@Override
public void writeToChannel(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes}));

View File

@ -57,6 +57,7 @@ public class DeltaLongEncodingWriter implements CompressionFactory.LongEncodingW
serializer.write(value - base);
}
@Override
public void putMeta(OutputStream metaOut, CompressedObjectStrategy.CompressionStrategy strategy) throws IOException
{
metaOut.write(CompressionFactory.setEncodingFlag(strategy.getId()));

View File

@ -75,16 +75,19 @@ public class IntermediateLongSupplierSerializer implements LongSupplierSerialize
this.compression = compression;
}
@Override
public void open() throws IOException
{
tempOut = new CountingOutputStream(ioPeon.makeOutputStream(tempFile));
}
@Override
public int size()
{
return numInserted;
}
@Override
public void add(long value) throws IOException
{
SerializerUtils.writeBigEndianLongToOutputStream(tempOut, value, helperBuffer);
@ -139,6 +142,7 @@ public class IntermediateLongSupplierSerializer implements LongSupplierSerialize
}
}
@Override
public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException
{
tempOut.close();
@ -146,6 +150,7 @@ public class IntermediateLongSupplierSerializer implements LongSupplierSerialize
delegate.closeAndConsolidate(consolidatedOut);
}
@Override
public void close() throws IOException
{
tempOut.close();
@ -153,11 +158,13 @@ public class IntermediateLongSupplierSerializer implements LongSupplierSerialize
delegate.close();
}
@Override
public long getSerializedSize()
{
return delegate.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
delegate.writeToChannel(channel, smoosher);

View File

@ -70,6 +70,7 @@ public class VSizeIndexedWriter extends MultiValueIndexedIntsWriter implements C
this.maxId = maxId;
}
@Override
public void open() throws IOException
{
headerOut = new CountingOutputStream(ioPeon.makeOutputStream(headerFileName));

View File

@ -497,6 +497,7 @@ public class VSizeLongSerde
this.offset = bufferOffset;
}
@Override
public long get(int index)
{
return buffer.getShort(offset + (index << 1)) & 0xFFFF;

View File

@ -180,6 +180,7 @@ public class OrFilter implements BooleanFilter
return Math.min(selectivity, 1.);
}
@Override
public String toString()
{
return String.format("(%s)", OR_JOINER.join(filters));

View File

@ -1002,11 +1002,13 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
this.value = value;
}
@Override
public TimeAndDims getKey()
{
return key;
}
@Override
public Integer getValue()
{
return value;
@ -1117,6 +1119,7 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
}
}
@Override
public Iterable<Map.Entry<TimeAndDims, Integer>> entrySet()
{
return facts.entrySet();
@ -1213,6 +1216,7 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
}
}
@Override
public Iterable<Map.Entry<TimeAndDims, Integer>> entrySet()
{
return concat(facts.values(), false);

View File

@ -84,6 +84,7 @@ public final class ProtoTestEventWrapper {
return defaultInstance;
}
@Override
public ProtoTestEvent getDefaultInstanceForType() {
return defaultInstance;
}
@ -99,7 +100,8 @@ public final class ProtoTestEventWrapper {
return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
@Override
protected GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable;
}
@ -116,6 +118,7 @@ public final class ProtoTestEventWrapper {
public static final int CATEGORY_TWO_VALUE = 2;
@Override
public final int getNumber() { return value; }
public static EventCategory valueOf(int value) {
@ -134,15 +137,18 @@ public final class ProtoTestEventWrapper {
private static com.google.protobuf.Internal.EnumLiteMap<EventCategory>
internalValueMap =
new com.google.protobuf.Internal.EnumLiteMap<EventCategory>() {
@Override
public EventCategory findValueByNumber(int number) {
return EventCategory.valueOf(number);
}
};
@Override
public final com.google.protobuf.Descriptors.EnumValueDescriptor
getValueDescriptor() {
return getDescriptor().getValues().get(index);
}
@Override
public final com.google.protobuf.Descriptors.EnumDescriptor
getDescriptorForType() {
return getDescriptor();
@ -180,9 +186,11 @@ public final class ProtoTestEventWrapper {
// required .prototest.ProtoTestEvent.EventCategory eventType = 1;
public static final int EVENTTYPE_FIELD_NUMBER = 1;
private ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_;
@Override
public boolean hasEventType() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
@Override
public ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() {
return eventType_;
}
@ -190,9 +198,11 @@ public final class ProtoTestEventWrapper {
// required uint64 id = 2;
public static final int ID_FIELD_NUMBER = 2;
private long id_;
@Override
public boolean hasId() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
@Override
public long getId() {
return id_;
}
@ -200,9 +210,11 @@ public final class ProtoTestEventWrapper {
// required string timestamp = 3;
public static final int TIMESTAMP_FIELD_NUMBER = 3;
private java.lang.Object timestamp_;
@Override
public boolean hasTimestamp() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
@Override
public String getTimestamp() {
java.lang.Object ref = timestamp_;
if (ref instanceof String) {
@ -232,9 +244,11 @@ public final class ProtoTestEventWrapper {
// optional uint32 someOtherId = 4;
public static final int SOMEOTHERID_FIELD_NUMBER = 4;
private int someOtherId_;
@Override
public boolean hasSomeOtherId() {
return ((bitField0_ & 0x00000008) == 0x00000008);
}
@Override
public int getSomeOtherId() {
return someOtherId_;
}
@ -242,9 +256,11 @@ public final class ProtoTestEventWrapper {
// optional bool isValid = 5;
public static final int ISVALID_FIELD_NUMBER = 5;
private boolean isValid_;
@Override
public boolean hasIsValid() {
return ((bitField0_ & 0x00000010) == 0x00000010);
}
@Override
public boolean getIsValid() {
return isValid_;
}
@ -252,9 +268,11 @@ public final class ProtoTestEventWrapper {
// optional string description = 6;
public static final int DESCRIPTION_FIELD_NUMBER = 6;
private java.lang.Object description_;
@Override
public boolean hasDescription() {
return ((bitField0_ & 0x00000020) == 0x00000020);
}
@Override
public String getDescription() {
java.lang.Object ref = description_;
if (ref instanceof String) {
@ -284,9 +302,11 @@ public final class ProtoTestEventWrapper {
// optional float someFloatColumn = 7;
public static final int SOMEFLOATCOLUMN_FIELD_NUMBER = 7;
private float someFloatColumn_;
@Override
public boolean hasSomeFloatColumn() {
return ((bitField0_ & 0x00000040) == 0x00000040);
}
@Override
public float getSomeFloatColumn() {
return someFloatColumn_;
}
@ -294,9 +314,11 @@ public final class ProtoTestEventWrapper {
// optional uint32 someIntColumn = 8;
public static final int SOMEINTCOLUMN_FIELD_NUMBER = 8;
private int someIntColumn_;
@Override
public boolean hasSomeIntColumn() {
return ((bitField0_ & 0x00000080) == 0x00000080);
}
@Override
public int getSomeIntColumn() {
return someIntColumn_;
}
@ -304,9 +326,11 @@ public final class ProtoTestEventWrapper {
// optional uint64 someLongColumn = 9;
public static final int SOMELONGCOLUMN_FIELD_NUMBER = 9;
private long someLongColumn_;
@Override
public boolean hasSomeLongColumn() {
return ((bitField0_ & 0x00000100) == 0x00000100);
}
@Override
public long getSomeLongColumn() {
return someLongColumn_;
}
@ -323,6 +347,7 @@ public final class ProtoTestEventWrapper {
someLongColumn_ = 0L;
}
private byte memoizedIsInitialized = -1;
@Override
public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized;
if (isInitialized != -1) {
@ -345,6 +370,7 @@ public final class ProtoTestEventWrapper {
return true;
}
@Override
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@ -379,6 +405,7 @@ public final class ProtoTestEventWrapper {
}
private int memoizedSerializedSize = -1;
@Override
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) {
@ -428,7 +455,7 @@ public final class ProtoTestEventWrapper {
}
private static final long serialVersionUID = 0L;
@java.lang.Override
@Override
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
@ -502,28 +529,31 @@ public final class ProtoTestEventWrapper {
}
public static Builder newBuilder() { return Builder.create(); }
@Override
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(ProtoTestEventWrapper.ProtoTestEvent prototype) {
return newBuilder().mergeFrom(prototype);
}
@Override
public Builder toBuilder() { return newBuilder(this); }
@SuppressWarnings("ParameterPackage")
@Override
protected Builder newBuilderForType(
com.google.protobuf.GeneratedMessage.BuilderParent parent) {
GeneratedMessage.BuilderParent parent) {
Builder builder = new Builder(parent);
return builder;
}
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder<Builder>
GeneratedMessage.Builder<Builder>
implements ProtoTestEventWrapper.ProtoTestEventOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
@Override
protected GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable;
}
@ -538,13 +568,14 @@ public final class ProtoTestEventWrapper {
maybeForceBuilderInitialization();
}
private void maybeForceBuilderInitialization() {
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
if (GeneratedMessage.alwaysUseFieldBuilders) {
}
}
private static Builder create() {
return new Builder();
}
@Override
public Builder clear() {
super.clear();
eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO;
@ -568,19 +599,23 @@ public final class ProtoTestEventWrapper {
return this;
}
@Override
public Builder clone() {
return create().mergeFrom(buildPartial());
}
@Override
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return ProtoTestEventWrapper.ProtoTestEvent.getDescriptor();
}
@Override
public ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() {
return ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance();
}
@Override
public ProtoTestEventWrapper.ProtoTestEvent build() {
ProtoTestEventWrapper.ProtoTestEvent result = buildPartial();
if (!result.isInitialized()) {
@ -600,6 +635,7 @@ public final class ProtoTestEventWrapper {
return result;
}
@Override
public ProtoTestEventWrapper.ProtoTestEvent buildPartial() {
ProtoTestEventWrapper.ProtoTestEvent result = new ProtoTestEventWrapper.ProtoTestEvent(this);
int from_bitField0_ = bitField0_;
@ -645,6 +681,7 @@ public final class ProtoTestEventWrapper {
return result;
}
@Override
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof ProtoTestEventWrapper.ProtoTestEvent) {
return mergeFrom((ProtoTestEventWrapper.ProtoTestEvent)other);
@ -689,6 +726,7 @@ public final class ProtoTestEventWrapper {
return this;
}
@Override
public final boolean isInitialized() {
if (!hasEventType()) {
@ -705,6 +743,7 @@ public final class ProtoTestEventWrapper {
return true;
}
@Override
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@ -787,9 +826,11 @@ public final class ProtoTestEventWrapper {
// required .prototest.ProtoTestEvent.EventCategory eventType = 1;
private ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO;
@Override
public boolean hasEventType() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
@Override
public ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() {
return eventType_;
}
@ -811,9 +852,11 @@ public final class ProtoTestEventWrapper {
// required uint64 id = 2;
private long id_ ;
@Override
public boolean hasId() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
@Override
public long getId() {
return id_;
}
@ -832,9 +875,11 @@ public final class ProtoTestEventWrapper {
// required string timestamp = 3;
private java.lang.Object timestamp_ = "";
@Override
public boolean hasTimestamp() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
@Override
public String getTimestamp() {
java.lang.Object ref = timestamp_;
if (!(ref instanceof String)) {
@ -868,9 +913,11 @@ public final class ProtoTestEventWrapper {
// optional uint32 someOtherId = 4;
private int someOtherId_ ;
@Override
public boolean hasSomeOtherId() {
return ((bitField0_ & 0x00000008) == 0x00000008);
}
@Override
public int getSomeOtherId() {
return someOtherId_;
}
@ -889,9 +936,11 @@ public final class ProtoTestEventWrapper {
// optional bool isValid = 5;
private boolean isValid_ ;
@Override
public boolean hasIsValid() {
return ((bitField0_ & 0x00000010) == 0x00000010);
}
@Override
public boolean getIsValid() {
return isValid_;
}
@ -910,9 +959,11 @@ public final class ProtoTestEventWrapper {
// optional string description = 6;
private java.lang.Object description_ = "";
@Override
public boolean hasDescription() {
return ((bitField0_ & 0x00000020) == 0x00000020);
}
@Override
public String getDescription() {
java.lang.Object ref = description_;
if (!(ref instanceof String)) {
@ -946,9 +997,11 @@ public final class ProtoTestEventWrapper {
// optional float someFloatColumn = 7;
private float someFloatColumn_ ;
@Override
public boolean hasSomeFloatColumn() {
return ((bitField0_ & 0x00000040) == 0x00000040);
}
@Override
public float getSomeFloatColumn() {
return someFloatColumn_;
}
@ -967,9 +1020,11 @@ public final class ProtoTestEventWrapper {
// optional uint32 someIntColumn = 8;
private int someIntColumn_ ;
@Override
public boolean hasSomeIntColumn() {
return ((bitField0_ & 0x00000080) == 0x00000080);
}
@Override
public int getSomeIntColumn() {
return someIntColumn_;
}
@ -988,9 +1043,11 @@ public final class ProtoTestEventWrapper {
// optional uint64 someLongColumn = 9;
private long someLongColumn_ ;
@Override
public boolean hasSomeLongColumn() {
return ((bitField0_ & 0x00000100) == 0x00000100);
}
@Override
public long getSomeLongColumn() {
return someLongColumn_;
}
@ -1021,7 +1078,7 @@ public final class ProtoTestEventWrapper {
private static com.google.protobuf.Descriptors.Descriptor
internal_static_prototest_ProtoTestEvent_descriptor;
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
GeneratedMessage.FieldAccessorTable
internal_static_prototest_ProtoTestEvent_fieldAccessorTable;
public static com.google.protobuf.Descriptors.FileDescriptor
@ -1045,13 +1102,14 @@ public final class ProtoTestEventWrapper {
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@Override
public com.google.protobuf.ExtensionRegistry assignDescriptors(
com.google.protobuf.Descriptors.FileDescriptor root) {
descriptor = root;
internal_static_prototest_ProtoTestEvent_descriptor =
getDescriptor().getMessageTypes().get(0);
internal_static_prototest_ProtoTestEvent_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
GeneratedMessage.FieldAccessorTable(
internal_static_prototest_ProtoTestEvent_descriptor,
new java.lang.String[] { "EventType", "Id", "Timestamp", "SomeOtherId", "IsValid", "Description", "SomeFloatColumn", "SomeIntColumn", "SomeLongColumn", },
ProtoTestEventWrapper.ProtoTestEvent.class,

View File

@ -165,6 +165,7 @@ public class RetryQueryRunnerTest
private int numTries = 1;
private boolean returnPartialResults = true;
@Override
public int getNumTries() { return numTries; }
public boolean returnPartialResults() { return returnPartialResults; }
@ -232,6 +233,7 @@ public class RetryQueryRunnerTest
private int numTries = 4;
private boolean returnPartialResults = true;
@Override
public int getNumTries() { return numTries; }
public boolean returnPartialResults() { return returnPartialResults; }
@ -284,6 +286,7 @@ public class RetryQueryRunnerTest
private int numTries = 1;
private boolean returnPartialResults = false;
@Override
public int getNumTries() { return numTries; }
public boolean returnPartialResults() { return returnPartialResults; }
@ -396,6 +399,7 @@ public class RetryQueryRunnerTest
private int numTries = 2;
private boolean returnPartialResults = false;
@Override
public int getNumTries() { return numTries; }
public boolean returnPartialResults() { return returnPartialResults; }

View File

@ -104,6 +104,7 @@ public class JavaScriptAggregatorBenchmark extends SimpleBenchmark
return floats[(int) (index % floats.length)];
}
@Override
public void increment()
{
++index;

View File

@ -53,6 +53,7 @@ public class CardinalityAggregatorBenchmark extends SimpleBenchmark
@Param({"true", "false"})
boolean byRow;
@Override
protected void setUp()
{
Iterable<String[]> values = FluentIterable

View File

@ -190,6 +190,7 @@ public class GroupByQueryMergeBufferTest
GroupByQueryRunnerTest.DEFAULT_MAPPER,
new GroupByQueryConfig()
{
@Override
public String getDefaultStrategy()
{
return "v2";

View File

@ -154,6 +154,7 @@ public class GroupByQueryRunnerFailureTest
GroupByQueryRunnerTest.DEFAULT_MAPPER,
new GroupByQueryConfig()
{
@Override
public String getDefaultStrategy()
{
return "v2";

View File

@ -136,6 +136,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
}
// GroupBy handles timestamps differently when granularity is ALL
@Override
@Test
public void testFullOnTimeseriesMaxMin()
{

View File

@ -110,6 +110,7 @@ public class LookupExtractorFactoryContainerTest
return null;
}
@Override
public boolean equals(Object other)
{
return other instanceof TestLookupExtractorFactory;

View File

@ -36,6 +36,7 @@ import java.util.Arrays;
public class CompressedVSizeIndexedV3SupplierTest extends CompressedVSizeIndexedSupplierTest
{
@Override
@Before
public void setUpSimple(){
vals = Arrays.asList(
@ -61,6 +62,7 @@ public class CompressedVSizeIndexedV3SupplierTest extends CompressedVSizeIndexed
);
}
@Override
@After
public void teardown(){
indexedSupplier = null;

View File

@ -169,6 +169,7 @@ public class BatchServerInventoryView extends ServerInventoryView<Set<DataSegmen
return container;
}
@Override
public void registerSegmentCallback(
final Executor exec,
final SegmentCallback callback,

Some files were not shown because too many files have changed in this diff Show More