mirror of https://github.com/apache/druid.git
Add GenericWhitespace checkstyle check (#5668)
This commit is contained in:
parent
ca3f833426
commit
a3a9ada843
|
@ -94,6 +94,7 @@
|
||||||
<property name="tokens" value="DOT"/>
|
<property name="tokens" value="DOT"/>
|
||||||
<property name="allowLineBreaks" value="false"/>
|
<property name="allowLineBreaks" value="false"/>
|
||||||
</module>
|
</module>
|
||||||
|
<module name="GenericWhitespace"/>
|
||||||
|
|
||||||
<module name="Indentation">
|
<module name="Indentation">
|
||||||
<property name="basicOffset" value="2"/>
|
<property name="basicOffset" value="2"/>
|
||||||
|
|
|
@ -182,7 +182,7 @@ public class DoublesSketchAggregatorFactory extends AggregatorFactory
|
||||||
@Override
|
@Override
|
||||||
public List<AggregatorFactory> getRequiredColumns()
|
public List<AggregatorFactory> getRequiredColumns()
|
||||||
{
|
{
|
||||||
return Collections.<AggregatorFactory> singletonList(
|
return Collections.singletonList(
|
||||||
new DoublesSketchAggregatorFactory(
|
new DoublesSketchAggregatorFactory(
|
||||||
fieldName,
|
fieldName,
|
||||||
fieldName,
|
fieldName,
|
||||||
|
|
|
@ -19,18 +19,17 @@
|
||||||
|
|
||||||
package io.druid.query.aggregation.datasketches.quantiles;
|
package io.druid.query.aggregation.datasketches.quantiles;
|
||||||
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.Module;
|
import com.fasterxml.jackson.databind.Module;
|
||||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||||
import com.google.inject.Binder;
|
import com.google.inject.Binder;
|
||||||
import com.yahoo.sketches.quantiles.DoublesSketch;
|
import com.yahoo.sketches.quantiles.DoublesSketch;
|
||||||
|
|
||||||
import io.druid.initialization.DruidModule;
|
import io.druid.initialization.DruidModule;
|
||||||
import io.druid.segment.serde.ComplexMetrics;
|
import io.druid.segment.serde.ComplexMetrics;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
public class DoublesSketchModule implements DruidModule
|
public class DoublesSketchModule implements DruidModule
|
||||||
{
|
{
|
||||||
|
|
||||||
|
@ -53,15 +52,17 @@ public class DoublesSketchModule implements DruidModule
|
||||||
@Override
|
@Override
|
||||||
public List<? extends Module> getJacksonModules()
|
public List<? extends Module> getJacksonModules()
|
||||||
{
|
{
|
||||||
return Arrays.<Module> asList(
|
return Collections.<Module>singletonList(
|
||||||
new SimpleModule("DoublesQuantilesSketchModule").registerSubtypes(
|
new SimpleModule("DoublesQuantilesSketchModule")
|
||||||
new NamedType(DoublesSketchAggregatorFactory.class, DOUBLES_SKETCH),
|
.registerSubtypes(
|
||||||
new NamedType(DoublesSketchMergeAggregatorFactory.class, DOUBLES_SKETCH_MERGE),
|
new NamedType(DoublesSketchAggregatorFactory.class, DOUBLES_SKETCH),
|
||||||
new NamedType(DoublesSketchToHistogramPostAggregator.class, DOUBLES_SKETCH_HISTOGRAM_POST_AGG),
|
new NamedType(DoublesSketchMergeAggregatorFactory.class, DOUBLES_SKETCH_MERGE),
|
||||||
new NamedType(DoublesSketchToQuantilePostAggregator.class, DOUBLES_SKETCH_QUANTILE_POST_AGG),
|
new NamedType(DoublesSketchToHistogramPostAggregator.class, DOUBLES_SKETCH_HISTOGRAM_POST_AGG),
|
||||||
new NamedType(DoublesSketchToQuantilesPostAggregator.class, DOUBLES_SKETCH_QUANTILES_POST_AGG),
|
new NamedType(DoublesSketchToQuantilePostAggregator.class, DOUBLES_SKETCH_QUANTILE_POST_AGG),
|
||||||
new NamedType(DoublesSketchToStringPostAggregator.class, DOUBLES_SKETCH_TO_STRING_POST_AGG))
|
new NamedType(DoublesSketchToQuantilesPostAggregator.class, DOUBLES_SKETCH_QUANTILES_POST_AGG),
|
||||||
.addSerializer(DoublesSketch.class, new DoublesSketchJsonSerializer()));
|
new NamedType(DoublesSketchToStringPostAggregator.class, DOUBLES_SKETCH_TO_STRING_POST_AGG)
|
||||||
|
).addSerializer(DoublesSketch.class, new DoublesSketchJsonSerializer())
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -636,7 +636,7 @@ public class TaskLockbox
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
final List<TaskLockPosse > possesHolder = dsRunning.get(interval);
|
final List<TaskLockPosse> possesHolder = dsRunning.get(interval);
|
||||||
if (possesHolder == null || possesHolder.isEmpty()) {
|
if (possesHolder == null || possesHolder.isEmpty()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
|
@ -203,7 +203,7 @@ public class OverlordResourceTest
|
||||||
expectAuthorizationTokenCheck();
|
expectAuthorizationTokenCheck();
|
||||||
|
|
||||||
List<String> tasksIds = ImmutableList.of("id_1", "id_2", "id_3");
|
List<String> tasksIds = ImmutableList.of("id_1", "id_2", "id_3");
|
||||||
EasyMock.<Collection<? extends TaskRunnerWorkItem>> expect(taskRunner.getRunningTasks()).andReturn(
|
EasyMock.<Collection<? extends TaskRunnerWorkItem>>expect(taskRunner.getRunningTasks()).andReturn(
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new MockTaskRunnerWorkItem(tasksIds.get(0), null),
|
new MockTaskRunnerWorkItem(tasksIds.get(0), null),
|
||||||
new MockTaskRunnerWorkItem(tasksIds.get(1), null),
|
new MockTaskRunnerWorkItem(tasksIds.get(1), null),
|
||||||
|
@ -373,7 +373,7 @@ public class OverlordResourceTest
|
||||||
{
|
{
|
||||||
|
|
||||||
List<String> tasksIds = ImmutableList.of("id_1", "id_2");
|
List<String> tasksIds = ImmutableList.of("id_1", "id_2");
|
||||||
EasyMock.<Collection<? extends TaskRunnerWorkItem>> expect(taskRunner.getRunningTasks()).andReturn(
|
EasyMock.<Collection<? extends TaskRunnerWorkItem>>expect(taskRunner.getRunningTasks()).andReturn(
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new MockTaskRunnerWorkItem(tasksIds.get(0), null),
|
new MockTaskRunnerWorkItem(tasksIds.get(0), null),
|
||||||
new MockTaskRunnerWorkItem(tasksIds.get(1), null)));
|
new MockTaskRunnerWorkItem(tasksIds.get(1), null)));
|
||||||
|
@ -398,7 +398,7 @@ public class OverlordResourceTest
|
||||||
expectAuthorizationTokenCheck();
|
expectAuthorizationTokenCheck();
|
||||||
|
|
||||||
List<String> tasksIds = ImmutableList.of("id_1", "id_2");
|
List<String> tasksIds = ImmutableList.of("id_1", "id_2");
|
||||||
EasyMock.<Collection<? extends TaskRunnerWorkItem>> expect(taskRunner.getRunningTasks()).andReturn(
|
EasyMock.<Collection<? extends TaskRunnerWorkItem>>expect(taskRunner.getRunningTasks()).andReturn(
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new MockTaskRunnerWorkItem(tasksIds.get(0), null),
|
new MockTaskRunnerWorkItem(tasksIds.get(0), null),
|
||||||
new MockTaskRunnerWorkItem(tasksIds.get(1), null)));
|
new MockTaskRunnerWorkItem(tasksIds.get(1), null)));
|
||||||
|
|
|
@ -113,7 +113,7 @@ public class Sequences
|
||||||
return new WrappingSequence<>(seq, wrapper);
|
return new WrappingSequence<>(seq, wrapper);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static <T> Sequence<T> withEffect(final Sequence <T> seq, final Runnable effect, final Executor exec)
|
public static <T> Sequence<T> withEffect(final Sequence<T> seq, final Runnable effect, final Executor exec)
|
||||||
{
|
{
|
||||||
// Uses YieldingSequenceBase to be able to execute the effect if all elements of the wrapped seq are processed
|
// Uses YieldingSequenceBase to be able to execute the effect if all elements of the wrapped seq are processed
|
||||||
// (i. e. it "is done"), but the yielder of the underlying seq throws some exception from close(). This logic could
|
// (i. e. it "is done"), but the yielder of the underlying seq throws some exception from close(). This logic could
|
||||||
|
|
|
@ -279,7 +279,7 @@ public class ConcatSequenceTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void testAll(Iterable <List<Integer>> vals) throws IOException
|
public void testAll(Iterable<List<Integer>> vals) throws IOException
|
||||||
{
|
{
|
||||||
final Iterable<TestSequence<Integer>> theSequences = Iterables.transform(
|
final Iterable<TestSequence<Integer>> theSequences = Iterables.transform(
|
||||||
vals,
|
vals,
|
||||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.query.filter;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -32,16 +31,16 @@ public class ColumnComparisonDimFilterTest
|
||||||
@Test
|
@Test
|
||||||
public void testGetCacheKey()
|
public void testGetCacheKey()
|
||||||
{
|
{
|
||||||
ColumnComparisonDimFilter columnComparisonDimFilter = new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec> of(
|
ColumnComparisonDimFilter columnComparisonDimFilter = new ColumnComparisonDimFilter(ImmutableList.of(
|
||||||
DefaultDimensionSpec.of("abc"),
|
DefaultDimensionSpec.of("abc"),
|
||||||
DefaultDimensionSpec.of("d")
|
DefaultDimensionSpec.of("d")
|
||||||
));
|
));
|
||||||
ColumnComparisonDimFilter columnComparisonDimFilter2 = new ColumnComparisonDimFilter(ImmutableList.<DimensionSpec> of(
|
ColumnComparisonDimFilter columnComparisonDimFilter2 = new ColumnComparisonDimFilter(ImmutableList.of(
|
||||||
DefaultDimensionSpec.of("d"),
|
DefaultDimensionSpec.of("d"),
|
||||||
DefaultDimensionSpec.of("abc")
|
DefaultDimensionSpec.of("abc")
|
||||||
));
|
));
|
||||||
ColumnComparisonDimFilter columnComparisonDimFilter3 = new ColumnComparisonDimFilter(
|
ColumnComparisonDimFilter columnComparisonDimFilter3 = new ColumnComparisonDimFilter(
|
||||||
ImmutableList.<DimensionSpec> of(
|
ImmutableList.of(
|
||||||
DefaultDimensionSpec.of("d"),
|
DefaultDimensionSpec.of("d"),
|
||||||
DefaultDimensionSpec.of("e")
|
DefaultDimensionSpec.of("e")
|
||||||
)
|
)
|
||||||
|
@ -61,19 +60,19 @@ public class ColumnComparisonDimFilterTest
|
||||||
public void testHashCode()
|
public void testHashCode()
|
||||||
{
|
{
|
||||||
ColumnComparisonDimFilter columnComparisonDimFilter = new ColumnComparisonDimFilter(
|
ColumnComparisonDimFilter columnComparisonDimFilter = new ColumnComparisonDimFilter(
|
||||||
ImmutableList.<DimensionSpec> of(
|
ImmutableList.of(
|
||||||
DefaultDimensionSpec.of("abc"),
|
DefaultDimensionSpec.of("abc"),
|
||||||
DefaultDimensionSpec.of("d")
|
DefaultDimensionSpec.of("d")
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
ColumnComparisonDimFilter columnComparisonDimFilter2 = new ColumnComparisonDimFilter(
|
ColumnComparisonDimFilter columnComparisonDimFilter2 = new ColumnComparisonDimFilter(
|
||||||
ImmutableList.<DimensionSpec> of(
|
ImmutableList.of(
|
||||||
DefaultDimensionSpec.of("d"),
|
DefaultDimensionSpec.of("d"),
|
||||||
DefaultDimensionSpec.of("abc")
|
DefaultDimensionSpec.of("abc")
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
ColumnComparisonDimFilter columnComparisonDimFilter3 = new ColumnComparisonDimFilter(
|
ColumnComparisonDimFilter columnComparisonDimFilter3 = new ColumnComparisonDimFilter(
|
||||||
ImmutableList.<DimensionSpec> of(
|
ImmutableList.of(
|
||||||
DefaultDimensionSpec.of("d"),
|
DefaultDimensionSpec.of("d"),
|
||||||
DefaultDimensionSpec.of("e")
|
DefaultDimensionSpec.of("e")
|
||||||
)
|
)
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
package io.druid.segment.realtime.appenderator;
|
package io.druid.segment.realtime.appenderator;
|
||||||
|
|
||||||
import io.druid.data.input.InputRow;
|
import io.druid.data.input.InputRow;
|
||||||
import io.druid.query.SegmentDescriptor;
|
|
||||||
import io.druid.segment.indexing.RealtimeTuningConfig;
|
import io.druid.segment.indexing.RealtimeTuningConfig;
|
||||||
import io.druid.segment.realtime.SegmentPublisher;
|
import io.druid.segment.realtime.SegmentPublisher;
|
||||||
import io.druid.segment.realtime.plumber.IntervalStartVersioningPolicy;
|
import io.druid.segment.realtime.plumber.IntervalStartVersioningPolicy;
|
||||||
|
@ -27,14 +26,12 @@ import io.druid.segment.realtime.plumber.NoopRejectionPolicyFactory;
|
||||||
import io.druid.segment.realtime.plumber.SegmentHandoffNotifier;
|
import io.druid.segment.realtime.plumber.SegmentHandoffNotifier;
|
||||||
import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
|
import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
|
||||||
import io.druid.server.coordination.DataSegmentAnnouncer;
|
import io.druid.server.coordination.DataSegmentAnnouncer;
|
||||||
import io.druid.timeline.DataSegment;
|
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.Executor;
|
|
||||||
|
|
||||||
public class AppenderatorPlumberTest
|
public class AppenderatorPlumberTest
|
||||||
{
|
{
|
||||||
|
@ -46,7 +43,7 @@ public class AppenderatorPlumberTest
|
||||||
this.appenderatorTester = new AppenderatorTester(10);
|
this.appenderatorTester = new AppenderatorTester(10);
|
||||||
DataSegmentAnnouncer segmentAnnouncer = EasyMock
|
DataSegmentAnnouncer segmentAnnouncer = EasyMock
|
||||||
.createMock(DataSegmentAnnouncer.class);
|
.createMock(DataSegmentAnnouncer.class);
|
||||||
segmentAnnouncer.announceSegment(EasyMock.<DataSegment> anyObject());
|
segmentAnnouncer.announceSegment(EasyMock.anyObject());
|
||||||
EasyMock.expectLastCall().anyTimes();
|
EasyMock.expectLastCall().anyTimes();
|
||||||
|
|
||||||
SegmentPublisher segmentPublisher = EasyMock
|
SegmentPublisher segmentPublisher = EasyMock
|
||||||
|
@ -62,9 +59,9 @@ public class AppenderatorPlumberTest
|
||||||
EasyMock
|
EasyMock
|
||||||
.expect(
|
.expect(
|
||||||
handoffNotifier.registerSegmentHandoffCallback(
|
handoffNotifier.registerSegmentHandoffCallback(
|
||||||
EasyMock.<SegmentDescriptor> anyObject(),
|
EasyMock.anyObject(),
|
||||||
EasyMock.<Executor> anyObject(),
|
EasyMock.anyObject(),
|
||||||
EasyMock.<Runnable> anyObject())).andReturn(true).anyTimes();
|
EasyMock.anyObject())).andReturn(true).anyTimes();
|
||||||
|
|
||||||
RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig(
|
RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig(
|
||||||
1,
|
1,
|
||||||
|
|
Loading…
Reference in New Issue