mirror of https://github.com/apache/druid.git
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:
parent
8333043b7b
commit
b7a52286e8
|
@ -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
|
||||
|
||||
|
|
|
@ -76,6 +76,7 @@ public class MapBasedRow implements Row
|
|||
return timestamp.getMillis();
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public DateTime getTimestamp()
|
||||
{
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -25,6 +25,7 @@ public class Runnables
|
|||
{
|
||||
public static Runnable getNoopRunnable(){
|
||||
return new Runnable(){
|
||||
@Override
|
||||
public void run(){}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -78,6 +78,7 @@ public class RectangularBound implements Bound
|
|||
return maxCoords;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getLimit()
|
||||
{
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -71,6 +71,7 @@ public class ExecsTest
|
|||
producer.submit(
|
||||
new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
for (int i = 0; i < nTasks; i++) {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -213,6 +213,7 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory<InputRowP
|
|||
|
||||
private final Runnable doNothingRunnable = new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -226,6 +226,7 @@ public class MapVirtualColumn implements VirtualColumn
|
|||
return valueDimension;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getOutputName()
|
||||
{
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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); }
|
||||
};
|
||||
|
||||
|
|
|
@ -107,6 +107,7 @@ public class ApproximateHistogramFoldingSerde extends ComplexMetricSerde
|
|||
return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ObjectStrategy getObjectStrategy()
|
||||
{
|
||||
return new ObjectStrategy<ApproximateHistogram>()
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -44,6 +44,7 @@ public final class CacheHandler implements AutoCloseable
|
|||
return cache;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
cacheManager.disposeCache(this);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -75,6 +75,7 @@ public class DetermineHashedPartitionsJob implements Jobby
|
|||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean run()
|
||||
{
|
||||
try {
|
||||
|
|
|
@ -104,6 +104,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean run()
|
||||
{
|
||||
try {
|
||||
|
|
|
@ -153,6 +153,7 @@ public class IndexGeneratorJob implements Jobby
|
|||
return jobStats;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean run()
|
||||
{
|
||||
try {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -47,6 +47,7 @@ public class LockAcquireAction implements TaskAction<TaskLock>
|
|||
return interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<TaskLock> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<TaskLock>()
|
||||
|
|
|
@ -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>>() {};
|
||||
|
|
|
@ -45,6 +45,7 @@ public class LockReleaseAction implements TaskAction<Void>
|
|||
return interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Void> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Void>() {};
|
||||
|
|
|
@ -46,6 +46,7 @@ public class LockTryAcquireAction implements TaskAction<TaskLock>
|
|||
return interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<TaskLock> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<TaskLock>()
|
||||
|
|
|
@ -56,6 +56,7 @@ public class SegmentInsertAction implements TaskAction<Set<DataSegment>>
|
|||
return segments;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Set<DataSegment>> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Set<DataSegment>>()
|
||||
|
|
|
@ -60,6 +60,7 @@ public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
|
|||
return interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<List<DataSegment>> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<List<DataSegment>>() {};
|
||||
|
|
|
@ -76,6 +76,7 @@ public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
|
|||
return intervals;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<List<DataSegment>> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<List<DataSegment>>() {};
|
||||
|
|
|
@ -51,6 +51,7 @@ public class SegmentMetadataUpdateAction implements TaskAction<Void>
|
|||
return segments;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Void> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Void>() {};
|
||||
|
|
|
@ -51,6 +51,7 @@ public class SegmentNukeAction implements TaskAction<Void>
|
|||
return segments;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Void> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Void>()
|
||||
|
|
|
@ -84,6 +84,7 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
|
|||
return endMetadata;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<SegmentPublishResult> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<SegmentPublishResult>()
|
||||
|
|
|
@ -147,6 +147,7 @@ public class HadoopIndexTask extends HadoopTask
|
|||
return spec;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public List<String> getHadoopDependencyCoordinates()
|
||||
{
|
||||
|
|
|
@ -516,6 +516,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@LifecycleStop
|
||||
public void stop()
|
||||
{
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -65,6 +65,7 @@ public class RemoteTaskRunnerRunPendingTasksConcurrencyTest
|
|||
remoteTaskRunner = rtrTestUtils.makeRemoteTaskRunner(
|
||||
new TestRemoteTaskRunnerConfig(new Period("PT3600S"))
|
||||
{
|
||||
@Override
|
||||
public int getPendingTasksRunnerNumThreads()
|
||||
{
|
||||
return 2;
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -58,6 +58,7 @@ public class FunctionalIterable<T> implements Iterable<T>
|
|||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<T> iterator()
|
||||
{
|
||||
return delegate.iterator();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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
|
||||
|
|
3
pom.xml
3
pom.xml
|
@ -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>
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -49,6 +49,7 @@ public class QueryDataSource implements DataSource
|
|||
return query;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() { return query.toString(); }
|
||||
|
||||
@Override
|
||||
|
|
|
@ -48,6 +48,7 @@ public class TableDataSource implements DataSource
|
|||
return Collections.singletonList(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() { return name; }
|
||||
|
||||
@Override
|
||||
|
|
|
@ -41,6 +41,7 @@ public interface Aggregator extends Closeable
|
|||
Object get();
|
||||
float getFloat();
|
||||
|
||||
@Override
|
||||
void close();
|
||||
|
||||
long getLong();
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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(); }
|
||||
};
|
||||
|
||||
|
|
|
@ -447,6 +447,7 @@ public class GroupByQueryEngine
|
|||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
// cleanup
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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 =
|
||||
|
|
|
@ -70,6 +70,7 @@ public class SearchSortSpec
|
|||
return ordering.getCacheKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return String.format("%sSort", ordering.toString());
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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}));
|
||||
|
|
|
@ -77,6 +77,7 @@ public class FloatMetricColumnSerializer implements MetricColumnSerializer
|
|||
closeFile(outFile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void closeFile(final File outFile) throws IOException
|
||||
{
|
||||
outFile.delete();
|
||||
|
|
|
@ -1075,6 +1075,7 @@ public class IndexMerger
|
|||
this.lastVal = NOT_INIT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int seek(int dictId)
|
||||
{
|
||||
if (dimConversions == null) {
|
||||
|
|
|
@ -81,6 +81,7 @@ public class LongMetricColumnSerializer implements MetricColumnSerializer
|
|||
closeFile(outFile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void closeFile(final File outFile) throws IOException
|
||||
{
|
||||
outFile.delete();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -181,6 +181,7 @@ public class ReferenceCountingSegment extends AbstractSegment
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T as(Class<T> clazz)
|
||||
{
|
||||
return getBaseSegment().as(clazz);
|
||||
|
|
|
@ -496,6 +496,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
|||
this.lastVal = NOT_INIT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int seek(int dictId)
|
||||
{
|
||||
if (dimConversions == null) {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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}));
|
||||
|
|
|
@ -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}));
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -497,6 +497,7 @@ public class VSizeLongSerde
|
|||
this.offset = bufferOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long get(int index)
|
||||
{
|
||||
return buffer.getShort(offset + (index << 1)) & 0xFFFF;
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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; }
|
||||
|
|
|
@ -104,6 +104,7 @@ public class JavaScriptAggregatorBenchmark extends SimpleBenchmark
|
|||
return floats[(int) (index % floats.length)];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void increment()
|
||||
{
|
||||
++index;
|
||||
|
|
|
@ -53,6 +53,7 @@ public class CardinalityAggregatorBenchmark extends SimpleBenchmark
|
|||
@Param({"true", "false"})
|
||||
boolean byRow;
|
||||
|
||||
@Override
|
||||
protected void setUp()
|
||||
{
|
||||
Iterable<String[]> values = FluentIterable
|
||||
|
|
|
@ -190,6 +190,7 @@ public class GroupByQueryMergeBufferTest
|
|||
GroupByQueryRunnerTest.DEFAULT_MAPPER,
|
||||
new GroupByQueryConfig()
|
||||
{
|
||||
@Override
|
||||
public String getDefaultStrategy()
|
||||
{
|
||||
return "v2";
|
||||
|
|
|
@ -154,6 +154,7 @@ public class GroupByQueryRunnerFailureTest
|
|||
GroupByQueryRunnerTest.DEFAULT_MAPPER,
|
||||
new GroupByQueryConfig()
|
||||
{
|
||||
@Override
|
||||
public String getDefaultStrategy()
|
||||
{
|
||||
return "v2";
|
||||
|
|
|
@ -136,6 +136,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
|
|||
}
|
||||
|
||||
// GroupBy handles timestamps differently when granularity is ALL
|
||||
@Override
|
||||
@Test
|
||||
public void testFullOnTimeseriesMaxMin()
|
||||
{
|
||||
|
|
|
@ -110,6 +110,7 @@ public class LookupExtractorFactoryContainerTest
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other)
|
||||
{
|
||||
return other instanceof TestLookupExtractorFactory;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue