Skip to content

Commit

Permalink
Update Error prone configuration; Fix bugs (apache#4252)
Browse files Browse the repository at this point in the history
* Make Errorprone the default compiler

* Address comments

* Make Error Prone's ClassCanBeStatic rule a error

* Preconditions allow only %s pattern

* Fix DruidCoordinatorBalancerTester

* Try to give the compiler more memory

* Remove distribution module activation on jdk 1.8 because only jdk 1.8 is used now

* Don't show compiler warnings

* Try different travis script

* Fix travis.yml

* Make Error Prone optional again

* For error-prone compiler

* Increase compiler's maxmem

* Don't run Error Prone for benchmarks because of OOM

* Skip install step in Travis

* Remove MetricHolder.writeToChannel()

* In travis.yml, check compilation before tests, because it may fail faster
  • Loading branch information
leventov authored and gianm committed May 12, 2017
1 parent e09e892 commit 1ebfa22
Show file tree
Hide file tree
Showing 69 changed files with 232 additions and 187 deletions.
5 changes: 4 additions & 1 deletion .travis.yml
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,10 @@ jdk:
- oraclejdk8

script:
- mvn test -B -Pparallel-test -Dmaven.fork.count=2 && mvn clean -Pstrict compile test-compile -B
- mvn clean -Pstrict -pl '!benchmarks' compile test-compile -B
- mvn test -B -Pparallel-test -Dmaven.fork.count=2

install: true

sudo: false

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -560,7 +560,7 @@ public List<Long> apply(Cursor input)
);
}

private class NoBitmapSelectorFilter extends SelectorFilter
private static class NoBitmapSelectorFilter extends SelectorFilter
{
public NoBitmapSelectorFilter(
String dimension,
Expand All @@ -577,7 +577,7 @@ public boolean supportsBitmapIndex(BitmapIndexSelector selector)
}
}

private class NoBitmapDimensionPredicateFilter extends DimensionPredicateFilter
private static class NoBitmapDimensionPredicateFilter extends DimensionPredicateFilter
{
public NoBitmapDimensionPredicateFilter(
final String dimension,
Expand All @@ -595,7 +595,7 @@ public boolean supportsBitmapIndex(BitmapIndexSelector selector)
}
}

private class NoBitmapSelectorDimFilter extends SelectorDimFilter
private static class NoBitmapSelectorDimFilter extends SelectorDimFilter
{
public NoBitmapSelectorDimFilter(
String dimension,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -285,7 +285,7 @@ public String generateNestedEvent() throws Exception
return mapper.writeValueAsString(wrapper);
}

public class BenchmarkEvent
public static class BenchmarkEvent
{

public String ts;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -384,7 +384,7 @@ public void testIntervalBasedTimeGeneration() throws Exception {
}


private class RowValueTracker
private static class RowValueTracker
{
private Map<String, Map<Object, Integer>> dimensionMap;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,7 @@ public ImmutableBitmap getImmutableBitmap()
return bitmapFactory.mapImmutableBitmap(tmpBuffer.asReadOnlyBuffer());
}

@SuppressWarnings("ArgumentParameterSwap")
public Iterable<ImmutableNode> getChildren()
{
return new Iterable<ImmutableNode>()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ public void testAdd()
String defaultKey = "defaultKey";
long actual;
actual = mapObject.add(defaultKey,defaultValue);
Assert.assertEquals("Values does not match", actual, defaultValue);
Assert.assertEquals("Values does not match", defaultValue, actual);
}

@Test
Expand All @@ -63,7 +63,6 @@ public void testSnapshot()
Assert.assertEquals("Maps size does not match",mapObject.size(),snapShotMap.size());
long expected = (long) snapShotMap.get(defaultKey);
AtomicLong actual = (AtomicLong) mapObject.get(defaultKey);
Assert.assertEquals("Values for key = " + defaultKey + " does not match",
actual.longValue(),expected);
Assert.assertEquals("Values for key = " + defaultKey + " does not match", expected, actual.longValue());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -308,7 +308,7 @@ public void close() throws IOException
/**
* Pull request.
*/
final class DruidPullRequest
static final class DruidPullRequest
{
private final MessageQueue messageQueue;
private final String tag;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ public OrcHadoopInputRowParser(
initialize();
}

@SuppressWarnings("ArgumentParameterSwap")
@Override
public InputRow parse(OrcStruct input)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ public QueryToolChest<ScanResultValue, ScanQuery> getToolchest()
return toolChest;
}

private class ScanQueryRunner implements QueryRunner<ScanResultValue>
private static class ScanQueryRunner implements QueryRunner<ScanResultValue>
{
private final ScanQueryEngine engine;
private final Segment segment;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,7 @@ public boolean equals(Object obj)
if (estimate != that.estimate ||
highBound != that.highBound ||
lowBound != that.lowBound ||
numStdDev != numStdDev) {
numStdDev != that.numStdDev) {
return false;
}
return true;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -548,19 +548,19 @@ protected ApproximateHistogram foldMin(
} else {
Preconditions.checkArgument(
mergedPositions.length >= maxSize,
"temp buffer [mergedPositions] too small: length must be at least [%d], got [%d]",
"temp buffer [mergedPositions] too small: length must be at least [%s], got [%s]",
maxSize,
mergedPositions.length
);
Preconditions.checkArgument(
mergedBins.length >= maxSize,
"temp buffer [mergedBins] too small: length must be at least [%d], got [%d]",
"temp buffer [mergedBins] too small: length must be at least [%s], got [%s]",
maxSize,
mergedPositions.length
);
Preconditions.checkArgument(
deltas.length >= maxSize,
"temp buffer [deltas] too small: length must be at least [%d], got [%d]",
"temp buffer [deltas] too small: length must be at least [%s], got [%s]",
maxSize,
mergedPositions.length
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ public KafkaIOConfig(
Preconditions.checkArgument(
endPartitions.getPartitionOffsetMap().get(partition) >= startPartitions.getPartitionOffsetMap()
.get(partition),
"end offset must be >= start offset for partition[%d]",
"end offset must be >= start offset for partition[%s]",
partition
);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,15 +60,15 @@

public class KafkaIndexTaskClient
{
public class NoTaskLocationException extends RuntimeException
public static class NoTaskLocationException extends RuntimeException
{
public NoTaskLocationException(String message)
{
super(message);
}
}

public class TaskNotRunnableException extends RuntimeException
public static class TaskNotRunnableException extends RuntimeException
{
public TaskNotRunnableException(String message)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ public class KafkaSupervisor implements Supervisor
* time, there should only be up to a maximum of [taskCount] actively-reading task groups (tracked in the [taskGroups]
* map) + zero or more pending-completion task groups (tracked in [pendingCompletionTaskGroups]).
*/
private class TaskGroup
private static class TaskGroup
{
// This specifies the partitions and starting offsets for this task group. It is set on group creation from the data
// in [partitionGroups] and never changes during the lifetime of this task group, which will live until a task in
Expand All @@ -150,7 +150,7 @@ Set<String> taskIds()
}
}

private class TaskData
private static class TaskData
{
TaskStatus status;
DateTime startTime;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@

public class KafkaSupervisorReport extends SupervisorReport
{
public class KafkaSupervisorReportPayload
public static class KafkaSupervisorReportPayload
{
private final String dataSource;
private final String topic;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -416,7 +416,7 @@ public void testDatasourceMetadata() throws Exception

KafkaIndexTask task = captured.getValue();
KafkaIOConfig taskConfig = task.getIOConfig();
Assert.assertEquals(String.format("sequenceName-0", DATASOURCE), taskConfig.getBaseSequenceName());
Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName());
Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0));
Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1));
Assert.assertEquals(30L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2));
Expand Down Expand Up @@ -1789,7 +1789,7 @@ private KafkaIndexTask createKafkaIndexTask(
);
}

private class TestTaskRunnerWorkItem extends TaskRunnerWorkItem
private static class TestTaskRunnerWorkItem extends TaskRunnerWorkItem
{

private TaskLocation location;
Expand All @@ -1807,7 +1807,7 @@ public TaskLocation getLocation()
}
}

private class TestableKafkaSupervisor extends KafkaSupervisor
private static class TestableKafkaSupervisor extends KafkaSupervisor
{
public TestableKafkaSupervisor(
TaskStorage taskStorage,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -158,7 +158,7 @@ public void testFailsToMoveMissing() throws Exception
), ImmutableMap.<String, Object>of("bucket", "DOES NOT EXIST", "baseKey", "baseKey2"));
}

private class MockStorageService extends RestS3Service {
private static class MockStorageService extends RestS3Service {
Map<String, Set<String>> storage = Maps.newHashMap();
boolean moved = false;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@
*/
public class S3DataSegmentPusherTest
{
private class ValueContainer<T> {
private static class ValueContainer<T> {
private T value;

public T getValue() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.indexer.HadoopDruidIndexerConfig;
Expand Down Expand Up @@ -74,13 +75,11 @@ public Job addInputPaths(HadoopDruidIndexerConfig config, Job job) throws IOExce
final FileSystem fs = betaInput.getFileSystem(job.getConfiguration());
final Granularity segmentGranularity = config.getGranularitySpec().getSegmentGranularity();

Map<Long, Long> inputModifiedTimes = new TreeMap<>(
Comparators.inverse(Comparators.comparable())
);
Map<Long, Long> inputModifiedTimes = new TreeMap<>(Ordering.natural().reverse());

for (FileStatus status : FSSpideringIterator.spiderIterable(fs, betaInput)) {
final DateTime key = segmentGranularity.toDate(status.getPath().toString());
final Long currVal = inputModifiedTimes.get(key);
final Long currVal = inputModifiedTimes.get(key.getMillis());
final long mTime = status.getModificationTime();

inputModifiedTimes.put(key.getMillis(), currVal == null ? mTime : Math.max(currVal, mTime));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,23 +67,23 @@ public static Collection<?> data(){
return Arrays.asList(
new Object[][]{
{
DetermineHashedPartitionsJobTest.class.getClass().getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(),
DetermineHashedPartitionsJobTest.class.getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(),
1L,
"2011-04-10T00:00:00.000Z/2011-04-11T00:00:00.000Z",
0,
1,
first
},
{
DetermineHashedPartitionsJobTest.class.getClass().getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(),
DetermineHashedPartitionsJobTest.class.getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(),
100L,
"2011-04-10T00:00:00.000Z/2011-04-16T00:00:00.000Z",
0,
6,
second
},
{
DetermineHashedPartitionsJobTest.class.getClass().getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(),
DetermineHashedPartitionsJobTest.class.getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(),
1L,
"2011-04-10T00:00:00.000Z/2011-04-16T00:00:00.000Z",
0,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ public class UtilsTest
@Rule
public TemporaryFolder tmpFolder = new TemporaryFolder();

private class CreateValueFromKey implements Function
private static class CreateValueFromKey implements Function
{
@Override public Object apply(Object input)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -310,8 +310,11 @@ public void onFailure(Throwable throwable)
waitingFor.decrement();
waitingForMonitor.notifyAll();
}
default:
break;
case CONNECTION_SUSPENDED:
case CONNECTION_RECONNECTED:
case CONNECTION_LOST:
// do nothing
}
}
}
Expand Down Expand Up @@ -1001,6 +1004,11 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th
retVal.setException(new IllegalStateException(message));
}
runPendingTasks();
break;
case CONNECTION_SUSPENDED:
case CONNECTION_RECONNECTED:
case CONNECTION_LOST:
// do nothing
}
}
catch (Exception e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -269,7 +269,7 @@ public void testResetSupervisor() throws Exception
verifyAll();
}

private class TestSupervisorSpec implements SupervisorSpec
private static class TestSupervisorSpec implements SupervisorSpec
{
private final String id;
private final Supervisor supervisor;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -307,7 +307,7 @@ public void testReset() throws Exception
verifyAll();
}

private class TestSupervisorSpec implements SupervisorSpec
private static class TestSupervisorSpec implements SupervisorSpec
{
private final String id;
private final Supervisor supervisor;
Expand Down
6 changes: 3 additions & 3 deletions integration-tests/src/main/java/org/testng/TestNG.java
Original file line number Diff line number Diff line change
Expand Up @@ -954,10 +954,10 @@ private void initializeCommandLineSuitesGroups()
}
}

private void addReporter(Class<? extends IReporter> r)
private void addReporter(Class<? extends IReporter> reporterClass)
{
if (!m_reporters.contains(r)) {
m_reporters.add(ClassHelper.newInstance(r));
if (m_reporters.stream().noneMatch(reporterClass::isInstance)) {
m_reporters.add(ClassHelper.newInstance(reporterClass));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -422,7 +422,7 @@ public void testCustomNestedPeriodFail()
}
}

private class PathDate
private static class PathDate
{
public final String path;
public final DateTime date;
Expand Down
Loading

0 comments on commit 1ebfa22

Please sign in to comment.