Catch some incorrect method parameter or call argument formatting patterns with checkstyle (#6461)

* Catch some incorrect method parameter or call argument formatting patterns with checkstyle

* Fix DiscoveryModule

* Inline parameters_and_arguments.txt

* Fix a bug in PolyBind

* Fix formatting
This commit is contained in:
Roman Leventov 2018-10-23 07:17:38 -03:00 committed by GitHub
parent c5bf4e7503
commit 84ac18dc1b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
341 changed files with 1576 additions and 1873 deletions

View File

@ -22,7 +22,6 @@ package org.apache.druid.benchmark;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Ints;
import org.apache.druid.java.util.common.guava.Accumulator;
import org.apache.druid.java.util.common.guava.MergeSequence;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
@ -94,25 +93,14 @@ public class MergeSequenceBenchmark
}
if (!toMerge.isEmpty()) {
partialMerged.add(new MergeSequence<Integer>(Ordering.natural(), Sequences.simple(toMerge)));
partialMerged.add(new MergeSequence<>(Ordering.natural(), Sequences.simple(toMerge)));
}
MergeSequence<Integer> mergeSequence = new MergeSequence(
Ordering.<Integer>natural(),
MergeSequence<Integer> mergeSequence = new MergeSequence<>(
Ordering.natural(),
Sequences.simple(partialMerged)
);
Integer accumulate = mergeSequence.accumulate(
0, new Accumulator<Integer, Integer>()
{
@Override
public Integer accumulate(Integer accumulated, Integer in)
{
return accumulated + in;
}
}
);
Integer accumulate = mergeSequence.accumulate(0, Integer::sum);
blackhole.consume(accumulate);
}
@Benchmark
@ -120,20 +108,8 @@ public class MergeSequenceBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void mergeFlat(final Blackhole blackhole)
{
MergeSequence<Integer> mergeSequence = new MergeSequence(Ordering.<Integer>natural(), Sequences.simple(sequences));
Integer accumulate = mergeSequence.accumulate(
0, new Accumulator<Integer, Integer>()
{
@Override
public Integer accumulate(Integer accumulated, Integer in)
{
return accumulated + in;
}
}
);
MergeSequence<Integer> mergeSequence = new MergeSequence<>(Ordering.natural(), Sequences.simple(sequences));
Integer accumulate = mergeSequence.accumulate(0, Integer::sum);
blackhole.consume(accumulate);
}
}

View File

@ -42,11 +42,8 @@ public class QueryBenchmarkUtil
Segment adapter
)
{
return new FinalizeResultsQueryRunner<T>(
new BySegmentQueryRunner<T>(
segmentId, adapter.getDataInterval().getStart(),
factory.createRunner(adapter)
),
return new FinalizeResultsQueryRunner<>(
new BySegmentQueryRunner<>(segmentId, adapter.getDataInterval().getStart(), factory.createRunner(adapter)),
(QueryToolChest<T, Query<T>>) factory.getToolchest()
);
}

View File

@ -50,7 +50,8 @@
<suppress checks="Indentation" files="[\\/]target[\\/]generated-test-sources[\\/]" />
<suppress checks="Indentation" files="ProtoTestEventWrapper.java" />
<suppress checks="Regex" files="ProtoTestEventWrapper.java" />
<suppress checks="OneStatementPerLine" files="[\\/]target[\\/]generated-test-sources[\\/]" />
<!-- extendedset is a fork of Alessandro Colantonio's CONCISE (COmpressed 'N' Composable Integer SEt) repository and licensed to ASF under a CLA is not true. -->

View File

@ -205,5 +205,63 @@
<property name="message" value="ObjectColumnSelector, LongColumnSelector, FloatColumnSelector
and DoubleColumnSelector must not be used in an instanceof statement, see Javadoc of those interfaces."/>
</module>
<module name="Regexp">
<property
name="format"
value='(?&lt;!ImmutableMap.of|Types.mapOf|orderedMap|makeSelectResults|makeListOfPairs)\(\s*\n +([^,\n\(\{"&lt;/]+|[^,\n\(\{" /]+&gt; [a-zA-Z0-9_]+)\, ++[^,\n/]+'
/>
<property name="illegalPattern" value="true"/>
<property name="message" value="&#10;
According to the Druid code style, if a method or constructor declaration or a call&#10;
doesn't fit a single line, each parameter or argument should be on it's own, e. g:&#10;
&#10;
MyReturnType myMethodWithAVeryLongName(&#10;
MyParamTypeA myParamAWithAVeryLongName,&#10;
MyParamTypeB myParamBWithAVeryLongName&#10;
)&#10;
&#10;
or&#10;
&#10;
MyValueType myValue = myMethodWithAVeryLongName(&#10;
myVeryLongArgA,&#10;
myVeryLongArgB&#10;
)&#10;
&#10;
The exceptions from this rule are map-like and pair-accepting constructors and methods,&#10;
for those it's preferred to put each pair on it's own line, e. g:&#10;
&#10;
Map&lt;MyKeyType, MyValueType&gt; myMap = ImmutableMap.of(&#10;
myKey1, myValue1,&#10;
myKey2, myValue2&#10;
)&#10;
&#10;
Always prefer to fit a declaration or a method or constructor call into a single line&#10;
(less than 120 cols), if possible.&#10;
&#10;
Two things to note to avoid unnecessary breakdown:&#10;
&#10;
1) Exceptions declared for a method could be broken to the next line separately, e. g:&#10;
&#10;
MyReturnType myMethodWithAVeryLongName(MyParamTypeA myParamA, MyParamTypeB myParamB)&#10;
throws MyExceptionTypeAWithVeryLongName, MyExceptionTypeBWithVeryLongName&#10;
&#10;
2) In a variable, field or constant assignment, it's often more readable to break the&#10;
whole right hand side expression to the next line, instead of breaking the expression&#10;
arguments, e. g:&#10;
&#10;
MyTypeWithAVeryLongName myVariableWithAVeryLongName =&#10;
myMethodWithAVeryLongName(myArgA, myArgB);&#10;
&#10;
Also note that this checkstyle rule (the one that caused this message to be printed)&#10;
doesn't spot all violations of the corresponding Druid code style rule. If you see&#10;
a place where method or constructor parameters or call arguments are not properly&#10;
located each on it's own line, but this checkstyle rule is silent, if doesn't mean&#10;
that the code is formatted correctly. Fix it anyway.&#10;
&#10;
If you encouter a map-like or a pair-accepting method that is reported by this&#10;
checkstyle rule, you should add it as an exception in the corresponding rule in&#10;
codestyle/checkstyle.xml.&#10;"/>
</module>
</module>
</module>

View File

@ -80,7 +80,10 @@ public class ConfigManager
poller = new PollingCallable();
ScheduledExecutors.scheduleWithFixedDelay(
exec, new Duration(0), config.get().getPollDuration().toStandardDuration(), poller
exec,
new Duration(0),
config.get().getPollDuration().toStandardDuration(),
poller
);
started = true;

View File

@ -72,9 +72,8 @@ public class CombiningSequence<T> implements Sequence<T>
@Override
public <OutType> Yielder<OutType> toYielder(OutType initValue, final YieldingAccumulator<OutType, T> accumulator)
{
final CombiningYieldingAccumulator<OutType, T> combiningAccumulator = new CombiningYieldingAccumulator<>(
ordering, mergeFn, accumulator
);
final CombiningYieldingAccumulator<OutType, T> combiningAccumulator =
new CombiningYieldingAccumulator<>(ordering, mergeFn, accumulator);
combiningAccumulator.setRetVal(initValue);
Yielder<T> baseYielder = baseSequence.toYielder(null, combiningAccumulator);

View File

@ -122,9 +122,7 @@ public class DimensionsSpec
@JsonIgnore
public List<SpatialDimensionSchema> getSpatialDimensions()
{
Iterable<NewSpatialDimensionSchema> filteredList = Iterables.filter(
dimensions, NewSpatialDimensionSchema.class
);
Iterable<NewSpatialDimensionSchema> filteredList = Iterables.filter(dimensions, NewSpatialDimensionSchema.class);
Iterable<SpatialDimensionSchema> transformedList = Iterables.transform(
filteredList,

View File

@ -91,12 +91,8 @@ public class DruidSecondaryModule implements Module
mapper.setInjectableValues(new GuiceInjectableValues(injector));
mapper.setAnnotationIntrospectors(
new AnnotationIntrospectorPair(
guiceIntrospector, mapper.getSerializationConfig().getAnnotationIntrospector()
),
new AnnotationIntrospectorPair(
guiceIntrospector, mapper.getDeserializationConfig().getAnnotationIntrospector()
)
new AnnotationIntrospectorPair(guiceIntrospector, mapper.getSerializationConfig().getAnnotationIntrospector()),
new AnnotationIntrospectorPair(guiceIntrospector, mapper.getDeserializationConfig().getAnnotationIntrospector())
);
}
}

View File

@ -34,6 +34,7 @@ import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.common.StringUtils;
import javax.annotation.Nullable;
import java.lang.annotation.Annotation;
import java.lang.reflect.ParameterizedType;
import java.util.Map;
import java.util.Properties;
@ -122,13 +123,10 @@ public class PolyBind
final TypeLiteral<T> interfaceType = interfaceKey.getTypeLiteral();
if (interfaceKey.getAnnotation() != null) {
return MapBinder.newMapBinder(
binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotation()
);
return MapBinder.newMapBinder(binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotation());
} else if (interfaceKey.getAnnotationType() != null) {
return MapBinder.newMapBinder(
binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotationType()
);
Class<? extends Annotation> annotationType = interfaceKey.getAnnotationType();
return MapBinder.newMapBinder(binder, TypeLiteral.get(String.class), interfaceType, annotationType);
} else {
return MapBinder.newMapBinder(binder, TypeLiteral.get(String.class), interfaceType);
}
@ -178,7 +176,7 @@ public class PolyBind
if (key.getAnnotation() != null) {
implsMap = (Map<String, Provider<T>>) injector.getInstance(Key.get(mapType, key.getAnnotation()));
} else if (key.getAnnotationType() != null) {
implsMap = (Map<String, Provider<T>>) injector.getInstance(Key.get(mapType, key.getAnnotation()));
implsMap = (Map<String, Provider<T>>) injector.getInstance(Key.get(mapType, key.getAnnotationType()));
} else {
implsMap = (Map<String, Provider<T>>) injector.getInstance(Key.get(mapType));
}

View File

@ -131,7 +131,12 @@ public class Execs
queue = new SynchronousQueue<>();
}
return new ThreadPoolExecutor(
1, 1, 0L, TimeUnit.MILLISECONDS, queue, makeThreadFactory(nameFormat, priority),
1,
1,
0L,
TimeUnit.MILLISECONDS,
queue,
makeThreadFactory(nameFormat, priority),
new RejectedExecutionHandler()
{
@Override

View File

@ -19,7 +19,6 @@
package org.apache.druid.java.util.common.concurrent;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
import org.apache.druid.java.util.common.logger.Logger;
import org.joda.time.Duration;
@ -180,8 +179,6 @@ public class ScheduledExecutors
public static ScheduledExecutorService fixed(int corePoolSize, String nameFormat)
{
return Executors.newScheduledThreadPool(
corePoolSize, new ThreadFactoryBuilder().setDaemon(true).setNameFormat(nameFormat).build()
);
return Executors.newScheduledThreadPool(corePoolSize, Execs.makeThreadFactory(nameFormat));
}
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.java.util.common.granularity;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.JsonSerializable;
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.jsontype.TypeSerializer;
@ -428,9 +427,8 @@ public class PeriodGranularity extends Granularity implements JsonSerializable
}
@Override
public void serialize(
JsonGenerator jsonGenerator, SerializerProvider serializerProvider
) throws IOException, JsonProcessingException
public void serialize(JsonGenerator jsonGenerator, SerializerProvider serializerProvider)
throws IOException
{
// Retain the same behavior as before #3850.
// i.e. when Granularity class was an enum.
@ -448,8 +446,10 @@ public class PeriodGranularity extends Granularity implements JsonSerializable
@Override
public void serializeWithType(
JsonGenerator jsonGenerator, SerializerProvider serializerProvider, TypeSerializer typeSerializer
) throws IOException, JsonProcessingException
JsonGenerator jsonGenerator,
SerializerProvider serializerProvider,
TypeSerializer typeSerializer
) throws IOException
{
serialize(jsonGenerator, serializerProvider);
}

View File

@ -48,9 +48,8 @@ public class FilteredSequence<T> implements Sequence<T>
@Override
public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
{
final FilteringYieldingAccumulator<OutType, T> filteringAccumulator = new FilteringYieldingAccumulator<>(
pred, accumulator
);
final FilteringYieldingAccumulator<OutType, T> filteringAccumulator =
new FilteringYieldingAccumulator<>(pred, accumulator);
return wrapYielder(baseSequence.toYielder(initValue, filteringAccumulator), filteringAccumulator);
}

View File

@ -82,9 +82,7 @@ public class JSONToLowerParser implements Parser<String, Object>
private ArrayList<String> fieldNames;
public JSONToLowerParser(
ObjectMapper objectMapper, Iterable<String> fieldNames, Iterable<String> exclude
)
public JSONToLowerParser(ObjectMapper objectMapper, Iterable<String> fieldNames, Iterable<String> exclude)
{
this.objectMapper = objectMapper;
if (fieldNames != null) {

View File

@ -71,9 +71,7 @@ public class FullResponseHandler implements HttpResponseHandler<FullResponseHold
}
@Override
public void exceptionCaught(
ClientResponse<FullResponseHolder> clientResponse, Throwable e
)
public void exceptionCaught(ClientResponse<FullResponseHolder> clientResponse, Throwable e)
{
// Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished
}

View File

@ -70,9 +70,7 @@ public class StatusResponseHandler implements HttpResponseHandler<StatusResponse
}
@Override
public void exceptionCaught(
ClientResponse<StatusResponseHolder> clientResponse, Throwable e
)
public void exceptionCaught(ClientResponse<StatusResponseHolder> clientResponse, Throwable e)
{
// Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished
}

View File

@ -313,7 +313,8 @@ public class SysMonitor extends FeedDefiningMonitor
}
if (du != null) {
final Map<String, Long> stats = diff.to(
name, ImmutableMap.<String, Long>builder()
name,
ImmutableMap.<String, Long>builder()
.put("sys/disk/read/size", du.getReadBytes())
.put("sys/disk/read/count", du.getReads())
.put("sys/disk/write/size", du.getWriteBytes())
@ -379,7 +380,8 @@ public class SysMonitor extends FeedDefiningMonitor
}
if (netstat != null) {
final Map<String, Long> stats = diff.to(
name, ImmutableMap.<String, Long>builder()
name,
ImmutableMap.<String, Long>builder()
.put("sys/net/read/size", netstat.getRxBytes())
.put("sys/net/read/packets", netstat.getRxPackets())
.put("sys/net/read/errors", netstat.getRxErrors())
@ -434,7 +436,8 @@ public class SysMonitor extends FeedDefiningMonitor
final Cpu cpu = cpus[i];
final String name = Integer.toString(i);
final Map<String, Long> stats = diff.to(
name, ImmutableMap.<String, Long>builder()
name,
ImmutableMap.<String, Long>builder()
.put("user", cpu.getUser()) // user = Δuser / Δtotal
.put("sys", cpu.getSys()) // sys = Δsys / Δtotal
.put("nice", cpu.getNice()) // nice = Δnice / Δtotal

View File

@ -61,7 +61,9 @@ public interface DataSegmentFinder
* @param segmentModifiedAt segment modified timestamp
*/
static void putInMapRetainingNewest(
Map<String, Pair<DataSegment, Long>> timestampedSegments, DataSegment dataSegment, long segmentModifiedAt
Map<String, Pair<DataSegment, Long>> timestampedSegments,
DataSegment dataSegment,
long segmentModifiedAt
)
{
timestampedSegments.merge(

View File

@ -20,7 +20,6 @@
package org.apache.druid;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.MapperFeature;
@ -55,12 +54,12 @@ public class TestObjectMapper extends ObjectMapper
{
addSerializer(Interval.class, ToStringSerializer.instance);
addDeserializer(
Interval.class, new StdDeserializer<Interval>(Interval.class)
Interval.class,
new StdDeserializer<Interval>(Interval.class)
{
@Override
public Interval deserialize(
JsonParser jsonParser, DeserializationContext deserializationContext
) throws IOException, JsonProcessingException
public Interval deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
throws IOException
{
return Intervals.of(jsonParser.getText());
}

View File

@ -57,7 +57,8 @@ public class UUIDUtilsTest
args.add(new String[]{possibleArg});
}
return Collections2.transform(
args, new Function<String[], Object[]>()
args,
new Function<String[], Object[]>()
{
@Override
public Object[] apply(String[] input)

View File

@ -65,7 +65,10 @@ public class JsonConfiguratorTest
@Override
public <T> Set<ConstraintViolation<T>> validateValue(
Class<T> beanType, String propertyName, Object value, Class<?>... groups
Class<T> beanType,
String propertyName,
Object value,
Class<?>... groups
)
{
return ImmutableSet.of();

View File

@ -24,17 +24,13 @@ package org.apache.druid.java.util.common.guava;
public class UnsupportedSequence implements Sequence<Integer>
{
@Override
public <OutType> OutType accumulate(
OutType initValue, Accumulator<OutType, Integer> accumulator
)
public <OutType> OutType accumulate(OutType initValue, Accumulator<OutType, Integer> accumulator)
{
throw new UnsupportedOperationException();
}
@Override
public <OutType> Yielder<OutType> toYielder(
OutType initValue, YieldingAccumulator<OutType, Integer> accumulator
)
public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, Integer> accumulator)
{
throw new UnsupportedOperationException();
}

View File

@ -246,7 +246,8 @@ public class FriendlyServersTest
final ListenableFuture<StatusResponseHolder> response2 = skepticalClient
.go(
new Request(
HttpMethod.GET, new URL(StringUtils.format("https://localhost:%d/", sslConnector.getLocalPort()))
HttpMethod.GET,
new URL(StringUtils.format("https://localhost:%d/", sslConnector.getLocalPort()))
),
new StatusResponseHandler(StandardCharsets.UTF_8)
);

View File

@ -122,12 +122,8 @@ public class EvalTest
Assert.assertFalse(evalLong("!9223372036854775807", bindings) > 0);
Assert.assertEquals(3037000499L, evalLong("cast(sqrt(9223372036854775807), 'long')", bindings));
Assert.assertEquals(
1L, evalLong("if(x == 9223372036854775807, 1, 0)", bindings)
);
Assert.assertEquals(
0L, evalLong("if(x - 1 == 9223372036854775807, 1, 0)", bindings)
);
Assert.assertEquals(1L, evalLong("if(x == 9223372036854775807, 1, 0)", bindings));
Assert.assertEquals(0L, evalLong("if(x - 1 == 9223372036854775807, 1, 0)", bindings));
Assert.assertEquals(1271030400000L, evalLong("timestamp('2010-04-12')", bindings));
Assert.assertEquals(1270998000000L, evalLong("timestamp('2010-04-12T+09:00')", bindings));

View File

@ -29,9 +29,7 @@ import java.util.Map;
public class NoopDataSegmentMover implements DataSegmentMover
{
@Override
public DataSegment move(
DataSegment segment, Map<String, Object> targetLoadSpec
)
public DataSegment move(DataSegment segment, Map<String, Object> targetLoadSpec)
{
return segment;
}

View File

@ -97,8 +97,7 @@ public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFireho
log.info("Retrieving file from region[%s], container[%s] and path [%s]",
region, container, path
);
CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy(
cloudFilesApi, region, container);
CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
return new CloudFilesByteSource(objectApi, path);
}

View File

@ -64,8 +64,10 @@ public class CloudFilesDataSegmentPuller
}
catch (IOException ioe) {
log.warn(
ioe, "Failed to remove output directory [%s] for segment pulled from [%s]",
outDir.getAbsolutePath(), path
ioe,
"Failed to remove output directory [%s] for segment pulled from [%s]",
outDir.getAbsolutePath(),
path
);
}
throw new SegmentLoadingException(e, e.getMessage());

View File

@ -92,7 +92,9 @@ public class CloudFilesDataSegmentPusher implements DataSegmentPusher
return CloudFilesUtils.retryCloudFilesOperation(
() -> {
CloudFilesObject segmentData = new CloudFilesObject(
segmentPath, outFile, objectApi.getRegion(),
segmentPath,
outFile,
objectApi.getRegion(),
objectApi.getContainer()
);
@ -103,8 +105,10 @@ public class CloudFilesDataSegmentPusher implements DataSegmentPusher
// runtime, and because Guava deletes methods over time, that causes incompatibilities.
Files.write(descFile.toPath(), jsonMapper.writeValueAsBytes(inSegment));
CloudFilesObject descriptorData = new CloudFilesObject(
segmentPath, descFile,
objectApi.getRegion(), objectApi.getContainer()
segmentPath,
descFile,
objectApi.getRegion(),
objectApi.getContainer()
);
log.info("Pushing %s.", descriptorData.getPath());
objectApi.put(descriptorData);

View File

@ -68,8 +68,10 @@ public class GoogleDataSegmentPuller implements URIDataPuller
}
catch (IOException ioe) {
LOG.warn(
ioe, "Failed to remove output directory [%s] for segment pulled from [%s]",
outDir.getAbsolutePath(), path
ioe,
"Failed to remove output directory [%s] for segment pulled from [%s]",
outDir.getAbsolutePath(),
path
);
}
throw new SegmentLoadingException(e, e.getMessage());

View File

@ -153,12 +153,14 @@ public class KafkaEightSimpleConsumerFirehoseFactory implements
final Map<Integer, Long> lastOffsets = loadOffsetFromPreviousMetaData(lastCommit);
for (Integer partition : partitionIdList) {
final KafkaSimpleConsumer kafkaSimpleConsumer = new KafkaSimpleConsumer(
feed, partition, clientId, brokerList, earliest
);
final KafkaSimpleConsumer kafkaSimpleConsumer =
new KafkaSimpleConsumer(feed, partition, clientId, brokerList, earliest);
Long startOffset = lastOffsets.get(partition);
PartitionConsumerWorker worker = new PartitionConsumerWorker(
feed, kafkaSimpleConsumer, partition, startOffset == null ? -1 : startOffset
feed,
kafkaSimpleConsumer,
partition,
startOffset == null ? -1 : startOffset
);
consumerWorkers.add(worker);
}

View File

@ -196,9 +196,7 @@ public class KafkaSimpleConsumer
earliest ? kafka.api.OffsetRequest.EarliestTime() : kafka.api.OffsetRequest.LatestTime(), 1
)
);
OffsetRequest request = new OffsetRequest(
requestInfo, kafka.api.OffsetRequest.CurrentVersion(), clientId
);
OffsetRequest request = new OffsetRequest(requestInfo, kafka.api.OffsetRequest.CurrentVersion(), clientId);
OffsetResponse response = null;
try {
response = consumer.getOffsetsBefore(request);
@ -323,7 +321,11 @@ public class KafkaSimpleConsumer
catch (Exception e) {
ensureNotInterrupted(e);
log.warn(
e, "error communicating with Kafka Broker [%s] to find leader for [%s] - [%s]", broker, topic, partitionId
e,
"error communicating with Kafka Broker [%s] to find leader for [%s] - [%s]",
broker,
topic,
partitionId
);
}
finally {

View File

@ -261,18 +261,14 @@ public class DatasourceOptimizerTest extends CuratorTestBase
private void setupViews() throws Exception
{
baseView = new BatchServerInventoryView(
zkPathsConfig,
curator,
jsonMapper,
Predicates.alwaysTrue()
)
baseView = new BatchServerInventoryView(zkPathsConfig, curator, jsonMapper, Predicates.alwaysTrue())
{
@Override
public void registerSegmentCallback(Executor exec, final SegmentCallback callback)
{
super.registerSegmentCallback(
exec, new SegmentCallback()
exec,
new SegmentCallback()
{
@Override
public CallbackAction segmentAdded(DruidServerMetadata server, DataSegment segment)

View File

@ -203,9 +203,7 @@ public class OrcIndexGeneratorJobTest
Map.class
),
aggs,
new UniformGranularitySpec(
Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)
),
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)),
null,
mapper
),

View File

@ -95,8 +95,10 @@ public class DruidParquetReadSupport extends AvroReadSupport<GenericRecord>
@Override
public RecordMaterializer<GenericRecord> prepareForRead(
Configuration configuration, Map<String, String> keyValueMetaData,
MessageType fileSchema, ReadContext readContext
Configuration configuration,
Map<String, String> keyValueMetaData,
MessageType fileSchema,
ReadContext readContext
)
{
MessageType parquetSchema = readContext.getRequestedSchema();

View File

@ -45,9 +45,8 @@ public class AvroValueInputFormat extends FileInputFormat<NullWritable, GenericR
* {@inheritDoc}
*/
@Override
public RecordReader<NullWritable, GenericRecord> createRecordReader(
InputSplit split, TaskAttemptContext context
) throws IOException
public RecordReader<NullWritable, GenericRecord> createRecordReader(InputSplit split, TaskAttemptContext context)
throws IOException
{
Schema readerSchema = AvroJob.getInputValueSchema(context.getConfiguration());

View File

@ -90,7 +90,9 @@ public class InlineSchemaAvroBytesDecoder implements AvroBytesDecoder
catch (EOFException eof) {
// waiting for avro v1.9.0 (#AVRO-813)
throw new ParseException(
eof, "Avro's unnecessary EOFException, detail: [%s]", "https://issues.apache.org/jira/browse/AVRO-813"
eof,
"Avro's unnecessary EOFException, detail: [%s]",
"https://issues.apache.org/jira/browse/AVRO-813"
);
}
catch (Exception e) {

View File

@ -122,7 +122,9 @@ public class InlineSchemasAvroBytesDecoder implements AvroBytesDecoder
catch (EOFException eof) {
// waiting for avro v1.9.0 (#AVRO-813)
throw new ParseException(
eof, "Avro's unnecessary EOFException, detail: [%s]", "https://issues.apache.org/jira/browse/AVRO-813"
eof,
"Avro's unnecessary EOFException, detail: [%s]",
"https://issues.apache.org/jira/browse/AVRO-813"
);
}
catch (Exception e) {

View File

@ -84,7 +84,9 @@ public class SchemaRepoBasedAvroBytesDecoder<SUBJECT, ID> implements AvroBytesDe
catch (EOFException eof) {
// waiting for avro v1.9.0 (#AVRO-813)
throw new ParseException(
eof, "Avro's unnecessary EOFException, detail: [%s]", "https://issues.apache.org/jira/browse/AVRO-813"
eof,
"Avro's unnecessary EOFException, detail: [%s]",
"https://issues.apache.org/jira/browse/AVRO-813"
);
}
catch (IOException e) {

View File

@ -281,7 +281,8 @@ public class AvroStreamInputRowParserTest
// towards Map avro field as druid dimension, need to convert its toString() back to HashMap to check equality
assertEquals(1, inputRow.getDimension("someIntValueMap").size());
assertEquals(
SOME_INT_VALUE_MAP_VALUE, new HashMap<CharSequence, Integer>(
SOME_INT_VALUE_MAP_VALUE,
new HashMap<CharSequence, Integer>(
Maps.transformValues(
Splitter.on(",")
.withKeyValueSeparator("=")
@ -299,7 +300,8 @@ public class AvroStreamInputRowParserTest
)
);
assertEquals(
SOME_STRING_VALUE_MAP_VALUE, new HashMap<CharSequence, CharSequence>(
SOME_STRING_VALUE_MAP_VALUE,
new HashMap<CharSequence, CharSequence>(
Splitter.on(",")
.withKeyValueSeparator("=")
.split(inputRow.getDimension("someIntValueMap").get(0).replaceAll("[\\{\\} ]", ""))

View File

@ -68,9 +68,7 @@ public class SketchModule implements DruidModule
new NamedType(SketchSetPostAggregator.class, THETA_SKETCH_SET_OP_POST_AGG),
new NamedType(SketchConstantPostAggregator.class, THETA_SKETCH_CONSTANT_POST_AGG)
)
.addSerializer(
SketchHolder.class, new SketchHolderJsonSerializer()
)
.addSerializer(SketchHolder.class, new SketchHolderJsonSerializer())
);
}
}

View File

@ -231,9 +231,7 @@ public class CommonCacheNotifier
}
@Override
public void exceptionCaught(
ClientResponse<StatusResponseHolder> clientResponse, Throwable e
)
public void exceptionCaught(ClientResponse<StatusResponseHolder> clientResponse, Throwable e)
{
// Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished
log.error(e, "exceptionCaught in CommonCacheNotifier ResponseHandler.");

View File

@ -152,9 +152,8 @@ public class BasicHTTPAuthenticator implements Authenticator
@Override
public void doFilter(
ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain
) throws IOException, ServletException
public void doFilter(ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain)
throws IOException, ServletException
{
HttpServletResponse httpResp = (HttpServletResponse) servletResponse;

View File

@ -68,9 +68,7 @@ public class BytesFullResponseHandler implements HttpResponseHandler<FullRespons
}
@Override
public void exceptionCaught(
ClientResponse<FullResponseHolder> clientResponse, Throwable e
)
public void exceptionCaught(ClientResponse<FullResponseHolder> clientResponse, Throwable e)
{
// Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished
}

View File

@ -67,9 +67,7 @@ public class BasicRoleBasedAuthorizer implements Authorizer
}
@Override
public Access authorize(
AuthenticationResult authenticationResult, Resource resource, Action action
)
public Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action)
{
if (authenticationResult == null) {
throw new IAE("WTF? authenticationResult should never be null.");

View File

@ -356,9 +356,7 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdater implements BasicAu
}
@Override
public void setPermissions(
String prefix, String roleName, List<ResourceAction> permissions
)
public void setPermissions(String prefix, String roleName, List<ResourceAction> permissions)
{
Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS));
setPermissionsInternal(prefix, roleName, permissions);

View File

@ -241,9 +241,7 @@ public class CoordinatorBasicAuthorizerResourceHandler implements BasicAuthorize
}
@Override
public Response setRolePermissions(
String authorizerName, String roleName, List<ResourceAction> permissions
)
public Response setRolePermissions(String authorizerName, String roleName, List<ResourceAction> permissions)
{
final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName);
if (authorizer == null) {

View File

@ -126,9 +126,7 @@ public class DefaultBasicAuthorizerResourceHandler implements BasicAuthorizerRes
}
@Override
public Response setRolePermissions(
String authorizerName, String roleName, List<ResourceAction> permissions
)
public Response setRolePermissions(String authorizerName, String roleName, List<ResourceAction> permissions)
{
return NOT_FOUND_RESPONSE;
}

View File

@ -40,9 +40,7 @@ public class BloomFilterSerializersModule extends SimpleModule
public BloomFilterSerializersModule()
{
registerSubtypes(
new NamedType(BloomDimFilter.class, BLOOM_FILTER_TYPE_NAME)
);
registerSubtypes(new NamedType(BloomDimFilter.class, BLOOM_FILTER_TYPE_NAME));
addSerializer(BloomKFilter.class, new BloomKFilterSerializer());
addDeserializer(BloomKFilter.class, new BloomKFilterDeserializer());
addDeserializer(BloomKFilterHolder.class, new BloomKFilterHolderDeserializer());
@ -50,16 +48,14 @@ public class BloomFilterSerializersModule extends SimpleModule
private static class BloomKFilterSerializer extends StdSerializer<BloomKFilter>
{
BloomKFilterSerializer()
{
super(BloomKFilter.class);
}
@Override
public void serialize(
BloomKFilter bloomKFilter, JsonGenerator jsonGenerator, SerializerProvider serializerProvider
) throws IOException
public void serialize(BloomKFilter bloomKFilter, JsonGenerator jsonGenerator, SerializerProvider serializerProvider)
throws IOException
{
jsonGenerator.writeBinary(bloomKFilterToBytes(bloomKFilter));
}
@ -67,16 +63,14 @@ public class BloomFilterSerializersModule extends SimpleModule
private static class BloomKFilterDeserializer extends StdDeserializer<BloomKFilter>
{
BloomKFilterDeserializer()
{
super(BloomKFilter.class);
}
@Override
public BloomKFilter deserialize(
JsonParser jsonParser, DeserializationContext deserializationContext
) throws IOException
public BloomKFilter deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
throws IOException
{
return bloomKFilterFromBytes(jsonParser.getBinaryValue());
}
@ -90,9 +84,8 @@ public class BloomFilterSerializersModule extends SimpleModule
}
@Override
public BloomKFilterHolder deserialize(
JsonParser jsonParser, DeserializationContext deserializationContext
) throws IOException
public BloomKFilterHolder deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
throws IOException
{
return BloomKFilterHolder.fromBytes(jsonParser.getBinaryValue());
}

View File

@ -91,8 +91,7 @@ public class DruidKerberosAuthenticationHandler extends KerberosAuthenticationHa
// specifically configured
final String[] spnegoPrincipals;
if ("*".equals(principal)) {
spnegoPrincipals = KerberosUtil.getPrincipalNames(
keytab, Pattern.compile("HTTP/.*"));
spnegoPrincipals = KerberosUtil.getPrincipalNames(keytab, Pattern.compile("HTTP/.*"));
if (spnegoPrincipals.length == 0) {
throw new ServletException("Principals do not exist in the keytab");
}

View File

@ -229,9 +229,8 @@ public class KerberosAuthenticator implements Authenticator
}
@Override
public void doFilter(
ServletRequest request, ServletResponse response, FilterChain filterChain
) throws IOException, ServletException
public void doFilter(ServletRequest request, ServletResponse response, FilterChain filterChain)
throws IOException, ServletException
{
HttpServletRequest httpReq = (HttpServletRequest) request;
@ -474,7 +473,9 @@ public class KerberosAuthenticator implements Authenticator
@Override
public void decorateProxyRequest(
HttpServletRequest clientRequest, HttpServletResponse proxyResponse, Request proxyRequest
HttpServletRequest clientRequest,
HttpServletResponse proxyResponse,
Request proxyRequest
)
{
Object cookieToken = clientRequest.getAttribute(SIGNED_TOKEN_ATTRIBUTE);
@ -674,8 +675,11 @@ public class KerberosAuthenticator implements Authenticator
* long, boolean, boolean)
*/
private static void tokenToAuthCookie(
HttpServletResponse resp, String token,
String domain, String path, long expires,
HttpServletResponse resp,
String token,
String domain,
String path,
long expires,
boolean isCookiePersistent,
boolean isSecure
)

View File

@ -378,7 +378,8 @@ public class HdfsDataSegmentPusherTest
addSerializer(Interval.class, ToStringSerializer.instance);
addSerializer(NumberedShardSpec.class, ToStringSerializer.instance);
addDeserializer(
Interval.class, new StdDeserializer<Interval>(Interval.class)
Interval.class,
new StdDeserializer<Interval>(Interval.class)
{
@Override
public Interval deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)

View File

@ -575,8 +575,15 @@ public class ApproximateHistogram
}
int mergedBinCount = combineBins(
this.binCount, this.positions, this.bins, h.binCount, h.positions, h.bins,
mergedPositions, mergedBins, deltas
this.binCount,
this.positions,
this.bins,
h.binCount,
h.positions,
h.bins,
mergedPositions,
mergedBins,
deltas
);
if (mergedBinCount == 0) {
return this;
@ -631,14 +638,26 @@ public class ApproximateHistogram
if (this.binCount + h.binCount <= this.size) {
// no need to merge bins
mergedBinCount = combineBins(
this.binCount, this.positions, this.bins,
h.binCount, h.positions, h.bins,
mergedPositions, mergedBins, null
this.binCount,
this.positions,
this.bins,
h.binCount,
h.positions,
h.bins,
mergedPositions,
mergedBins,
null
);
} else {
mergedBinCount = ruleCombineBins(
this.binCount, this.positions, this.bins, h.binCount, h.positions, h.bins,
mergedPositions, mergedBins
this.binCount,
this.positions,
this.bins,
h.binCount,
h.positions,
h.bins,
mergedPositions,
mergedBins
);
}
for (int i = 0; i < mergedBinCount; ++i) {
@ -653,9 +672,14 @@ public class ApproximateHistogram
}
protected int ruleCombineBins(
int leftBinCount, float[] leftPositions, long[] leftBins,
int rightBinCount, float[] rightPositions, long[] rightBins,
float[] mergedPositions, long[] mergedBins
int leftBinCount,
float[] leftPositions,
long[] leftBins,
int rightBinCount,
float[] rightPositions,
long[] rightBins,
float[] mergedPositions,
long[] mergedBins
)
{
final float cutoff;
@ -855,7 +879,8 @@ public class ApproximateHistogram
* @return the last valid index into the mergedPositions and mergedBins arrays
*/
private static void mergeBins(
int mergedBinCount, float[] mergedPositions,
int mergedBinCount,
float[] mergedPositions,
long[] mergedBins,
float[] deltas,
int numMerge,
@ -1049,9 +1074,15 @@ public class ApproximateHistogram
* @return the number of combined bins
*/
private static int combineBins(
int leftBinCount, float[] leftPositions, long[] leftBins,
int rightBinCount, float[] rightPositions, long[] rightBins,
float[] mergedPositions, long[] mergedBins, float[] deltas
int leftBinCount,
float[] leftPositions,
long[] leftBins,
int rightBinCount,
float[] rightPositions,
long[] rightBins,
float[] mergedPositions,
long[] mergedBins,
float[] deltas
)
{
int i = 0;

View File

@ -242,7 +242,8 @@ public class KafkaLookupExtractorFactory implements LookupExtractorFactory
}
);
Futures.addCallback(
future, new FutureCallback<Object>()
future,
new FutureCallback<Object>()
{
@Override
public void onSuccess(Object result)

View File

@ -95,7 +95,10 @@ public class KafkaLookupExtractorFactoryTest
{
@Override
public Object findInjectableValue(
Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
Object valueId,
DeserializationContext ctxt,
BeanProperty forProperty,
Object beanInstance
)
{
if ("org.apache.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager".equals(valueId)) {

View File

@ -67,7 +67,11 @@ public class KafkaIndexTaskClient extends IndexTaskClient
try {
final FullResponseHolder response = submitRequestWithEmptyContent(
id, HttpMethod.POST, "stop", publish ? "publish=true" : null, true
id,
HttpMethod.POST,
"stop",
publish ? "publish=true" : null,
true
);
return isSuccess(response);
}

View File

@ -2195,9 +2195,9 @@ public class KafkaSupervisor implements Supervisor
Map<Integer, Long> currentOffsets = entry.getValue().currentOffsets;
Long remainingSeconds = null;
if (startTime != null) {
remainingSeconds = Math.max(
0, ioConfig.getTaskDuration().getMillis() - (System.currentTimeMillis() - startTime.getMillis())
) / 1000;
long elapsedMillis = System.currentTimeMillis() - startTime.getMillis();
long remainingMillis = Math.max(0, ioConfig.getTaskDuration().getMillis() - elapsedMillis);
remainingSeconds = TimeUnit.MILLISECONDS.toSeconds(remainingMillis);
}
taskReports.add(

View File

@ -2102,9 +2102,7 @@ public class KafkaIndexTaskTest
)
{
@Override
public <T> QueryRunner<T> decorate(
QueryRunner<T> delegate, QueryToolChest<T, ? extends Query<T>> toolChest
)
public <T> QueryRunner<T> decorate(QueryRunner<T> delegate, QueryToolChest<T, ? extends Query<T>> toolChest)
{
return delegate;
}
@ -2204,7 +2202,9 @@ public class KafkaIndexTaskTest
{
@Override
public boolean registerSegmentHandoffCallback(
SegmentDescriptor descriptor, Executor exec, Runnable handOffRunnable
SegmentDescriptor descriptor,
Executor exec,
Runnable handOffRunnable
)
{
if (doHandoff) {

View File

@ -134,9 +134,7 @@ public class KafkaSupervisorIOConfigTest
+ " \"ssl.key.password\":\"mykeypassword\"}\n"
+ "}";
KafkaSupervisorIOConfig config = mapper.readValue(
jsonStr, KafkaSupervisorIOConfig.class
);
KafkaSupervisorIOConfig config = mapper.readValue(jsonStr, KafkaSupervisorIOConfig.class);
Properties props = new Properties();
KafkaIndexTask.addConsumerPropertiesFromConfig(props, mapper, config.getConsumerProperties());

View File

@ -91,7 +91,10 @@ public class NamespaceLookupExtractorFactoryTest
{
@Override
public Object findInjectableValue(
Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
Object valueId,
DeserializationContext ctxt,
BeanProperty forProperty,
Object beanInstance
)
{
if (CacheScheduler.class.getName().equals(valueId)) {
@ -524,7 +527,9 @@ public class NamespaceLookupExtractorFactoryTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, false, null, null, true, false)
binder,
Key.get(DruidNode.class, Self.class),
new DruidNode("test-inject", null, false, null, null, true, false)
);
}
}

View File

@ -68,10 +68,12 @@ public class UriExtractionNamespaceTest
final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector();
mapper.setAnnotationIntrospectors(
new AnnotationIntrospectorPair(
guiceIntrospector, mapper.getSerializationConfig().getAnnotationIntrospector()
guiceIntrospector,
mapper.getSerializationConfig().getAnnotationIntrospector()
),
new AnnotationIntrospectorPair(
guiceIntrospector, mapper.getDeserializationConfig().getAnnotationIntrospector()
guiceIntrospector,
mapper.getDeserializationConfig().getAnnotationIntrospector()
)
);
return mapper;

View File

@ -49,7 +49,9 @@ public class OffHeapNamespaceExtractionCacheManagerTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, false, null, null, true, false)
binder,
Key.get(DruidNode.class, Self.class),
new DruidNode("test-inject", null, false, null, null, true, false)
);
}
}

View File

@ -49,7 +49,9 @@ public class OnHeapNamespaceExtractionCacheManagerTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, false, null, null, true, false)
binder,
Key.get(DruidNode.class, Self.class),
new DruidNode("test-inject", null, false, null, null, true, false)
);
}
}

View File

@ -39,10 +39,7 @@ public class LookupCacheStats
/**
* Constructs a new {@code CacheStats} instance.
*/
public LookupCacheStats(
long hitCount, long missCount,
long evictionCount
)
public LookupCacheStats(long hitCount, long missCount, long evictionCount)
{
Preconditions.checkArgument(hitCount >= 0);
Preconditions.checkArgument(missCount >= 0);

View File

@ -62,7 +62,8 @@ public class OffHeapPollingCache<K, V> implements PollingCache<K, V>
final Set<V> setOfValues = setOfValuesBuilder.build();
reverseCache.putAll(Maps.asMap(
setOfValues, new Function<V, List<K>>()
setOfValues,
new Function<V, List<K>>()
{
@Override
public List<K> apply(final V input)
@ -80,7 +81,8 @@ public class OffHeapPollingCache<K, V> implements PollingCache<K, V>
}
}).keySet());
}
}));
}
));
started.getAndSet(true);
}
}

View File

@ -93,7 +93,8 @@ public class S3DataSegmentMover implements DataSegmentMover
ImmutableMap.<String, Object>builder()
.putAll(
Maps.filterKeys(
loadSpec, new Predicate<String>()
loadSpec,
new Predicate<String>()
{
@Override
public boolean apply(String input)

View File

@ -40,16 +40,22 @@ import java.util.Map;
public class S3DataSegmentArchiverTest
{
private static final ObjectMapper MAPPER = new DefaultObjectMapper()
.setInjectableValues(new InjectableValues()
{
@Override
public Object findInjectableValue(
Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
)
{
return PULLER;
}
}).registerModule(new SimpleModule("s3-archive-test-module").registerSubtypes(S3LoadSpec.class));
.setInjectableValues(
new InjectableValues()
{
@Override
public Object findInjectableValue(
Object valueId,
DeserializationContext ctxt,
BeanProperty forProperty,
Object beanInstance
)
{
return PULLER;
}
}
)
.registerModule(new SimpleModule("s3-archive-test-module").registerSubtypes(S3LoadSpec.class));
private static final S3DataSegmentArchiverConfig ARCHIVER_CONFIG = new S3DataSegmentArchiverConfig()
{
@Override

View File

@ -50,8 +50,10 @@ public class PvaluefromZscorePostAggregatorTest
public void testSerde() throws Exception
{
DefaultObjectMapper mapper = new DefaultObjectMapper();
PvaluefromZscorePostAggregator postAggregator1 = mapper.readValue(mapper.writeValueAsString(
pvaluefromZscorePostAggregator), PvaluefromZscorePostAggregator.class);
PvaluefromZscorePostAggregator postAggregator1 = mapper.readValue(
mapper.writeValueAsString(pvaluefromZscorePostAggregator),
PvaluefromZscorePostAggregator.class
);
Assert.assertEquals(pvaluefromZscorePostAggregator, postAggregator1);
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.hll;
import com.google.common.base.Function;
import com.google.common.collect.Collections2;
import com.google.common.collect.Lists;
import com.google.common.hash.HashFunction;
@ -31,7 +30,6 @@ import org.junit.Assert;
import org.junit.Ignore;
import org.junit.Test;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.security.MessageDigest;
import java.util.Arrays;
@ -763,19 +761,8 @@ public class HyperLogLogCollectorTest
);
List<HyperLogLogCollector> collectors = Lists.transform(
objects, new Function<String, HyperLogLogCollector>()
{
@Nullable
@Override
public HyperLogLogCollector apply(
@Nullable String s
)
{
return HyperLogLogCollector.makeCollector(
ByteBuffer.wrap(Base64.decodeBase64(s))
);
}
}
objects,
s -> HyperLogLogCollector.makeCollector(ByteBuffer.wrap(Base64.decodeBase64(s)))
);
Collection<List<HyperLogLogCollector>> permutations = Collections2.permutations(collectors);

View File

@ -167,7 +167,7 @@ public class DetermineHashedPartitionsJob implements Jobby
if (Utils.exists(groupByJob, fileSystem, partitionInfoPath)) {
final Long numRows = config.JSON_MAPPER.readValue(
Utils.openInputStream(groupByJob, partitionInfoPath),
new TypeReference<Long>() {}
Long.class
);
log.info("Found approximately [%,d] rows in data.", numRows);
@ -381,19 +381,12 @@ public class DetermineHashedPartitionsJob implements Jobby
intervals.add(interval);
final Path outPath = config.makeSegmentPartitionInfoPath(interval);
final OutputStream out = Utils.makePathAndOutputStream(
context, outPath, config.isOverwriteFiles()
);
final OutputStream out = Utils.makePathAndOutputStream(context, outPath, config.isOverwriteFiles());
try {
HadoopDruidIndexerConfig.JSON_MAPPER.writerWithType(
new TypeReference<Long>()
{
}
).writeValue(
out,
aggregate.estimateCardinalityRound()
);
HadoopDruidIndexerConfig.JSON_MAPPER
.writerWithType(Long.class)
.writeValue(out, aggregate.estimateCardinalityRound());
}
finally {
Closeables.close(out, false);
@ -407,9 +400,7 @@ public class DetermineHashedPartitionsJob implements Jobby
super.run(context);
if (determineIntervals) {
final Path outPath = config.makeIntervalInfoPath();
final OutputStream out = Utils.makePathAndOutputStream(
context, outPath, config.isOverwriteFiles()
);
final OutputStream out = Utils.makePathAndOutputStream(context, outPath, config.isOverwriteFiles());
try {
HadoopDruidIndexerConfig.JSON_MAPPER.writerWithType(

View File

@ -360,9 +360,7 @@ public class DeterminePartitionsJob implements Jobby
}
@Override
protected void map(
BytesWritable key, NullWritable value, Context context
) throws IOException, InterruptedException
protected void map(BytesWritable key, NullWritable value, Context context) throws IOException, InterruptedException
{
final List<Object> timeAndDims = HadoopDruidIndexerConfig.JSON_MAPPER.readValue(key.getBytes(), List.class);
@ -529,9 +527,8 @@ public class DeterminePartitionsJob implements Jobby
}
@Override
protected void reduce(
BytesWritable key, Iterable<Text> values, Context context
) throws IOException, InterruptedException
protected void reduce(BytesWritable key, Iterable<Text> values, Context context)
throws IOException, InterruptedException
{
SortableBytes keyBytes = SortableBytes.fromBytesWritable(key);
@ -540,7 +537,9 @@ public class DeterminePartitionsJob implements Jobby
}
protected abstract void innerReduce(
Context context, SortableBytes keyBytes, Iterable<DimValueCount> combinedIterable
Context context,
SortableBytes keyBytes,
Iterable<DimValueCount> combinedIterable
) throws IOException, InterruptedException;
private Iterable<DimValueCount> combineRows(Iterable<Text> input)
@ -586,9 +585,8 @@ public class DeterminePartitionsJob implements Jobby
public static class DeterminePartitionsDimSelectionCombiner extends DeterminePartitionsDimSelectionBaseReducer
{
@Override
protected void innerReduce(
Context context, SortableBytes keyBytes, Iterable<DimValueCount> combinedIterable
) throws IOException, InterruptedException
protected void innerReduce(Context context, SortableBytes keyBytes, Iterable<DimValueCount> combinedIterable)
throws IOException, InterruptedException
{
for (DimValueCount dvc : combinedIterable) {
write(context, keyBytes.getGroupKey(), dvc);
@ -602,9 +600,8 @@ public class DeterminePartitionsJob implements Jobby
private static final int HIGH_CARDINALITY_THRESHOLD = 3000000;
@Override
protected void innerReduce(
Context context, SortableBytes keyBytes, Iterable<DimValueCount> combinedIterable
) throws IOException
protected void innerReduce(Context context, SortableBytes keyBytes, Iterable<DimValueCount> combinedIterable)
throws IOException
{
final ByteBuffer groupKey = ByteBuffer.wrap(keyBytes.getGroupKey());
groupKey.position(4); // Skip partition
@ -807,14 +804,8 @@ public class DeterminePartitionsJob implements Jobby
: minDistancePartitions;
final List<ShardSpec> chosenShardSpecs = Lists.transform(
chosenPartitions.partitions, new Function<DimPartition, ShardSpec>()
{
@Override
public ShardSpec apply(DimPartition dimPartition)
{
return dimPartition.shardSpec;
}
}
chosenPartitions.partitions,
dimPartition -> dimPartition.shardSpec
);
log.info("Chosen partitions:");
@ -950,13 +941,8 @@ public class DeterminePartitionsJob implements Jobby
)
throws IOException, InterruptedException
{
context.write(
new SortableBytes(
groupKey, TAB_JOINER.join(dimValueCount.dim, dimValueCount.value).getBytes(
HadoopDruidIndexerConfig.JAVA_NATIVE_CHARSET
)
).toBytesWritable(),
dimValueCount.toText()
);
byte[] sortKey = TAB_JOINER.join(dimValueCount.dim, dimValueCount.value)
.getBytes(HadoopDruidIndexerConfig.JAVA_NATIVE_CHARSET);
context.write(new SortableBytes(groupKey, sortKey).toBytesWritable(), dimValueCount.toText());
}
}

View File

@ -109,7 +109,9 @@ public class HadoopDruidIndexerConfig
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("hadoop-indexer", null, false, null, null, true, false)
binder,
Key.get(DruidNode.class, Self.class),
new DruidNode("hadoop-indexer", null, false, null, null, true, false)
);
JsonConfigProvider.bind(binder, "druid.hadoop.security.kerberos", HadoopKerberosConfig.class);
}
@ -162,9 +164,7 @@ public class HadoopDruidIndexerConfig
{
try {
return fromMap(
HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
file, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
)
HadoopDruidIndexerConfig.JSON_MAPPER.readValue(file, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT)
);
}
catch (IOException e) {
@ -178,9 +178,7 @@ public class HadoopDruidIndexerConfig
// This is a map to try and prevent dependency screwbally-ness
try {
return fromMap(
HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
str, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
)
HadoopDruidIndexerConfig.JSON_MAPPER.readValue(str, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT)
);
}
catch (IOException e) {
@ -197,9 +195,7 @@ public class HadoopDruidIndexerConfig
Reader reader = new InputStreamReader(fs.open(pt), StandardCharsets.UTF_8);
return fromMap(
HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
reader, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
)
HadoopDruidIndexerConfig.JSON_MAPPER.readValue(reader, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT)
);
}
catch (Exception e) {

View File

@ -420,11 +420,9 @@ public class IndexGeneratorJob implements Jobby
}
@Override
protected void reduce(
final BytesWritable key, Iterable<BytesWritable> values, final Context context
) throws IOException, InterruptedException
protected void reduce(final BytesWritable key, Iterable<BytesWritable> values, final Context context)
throws IOException, InterruptedException
{
Iterator<BytesWritable> iter = values.iterator();
BytesWritable first = iter.next();
@ -587,9 +585,8 @@ public class IndexGeneratorJob implements Jobby
final ProgressIndicator progressIndicator
) throws IOException
{
return HadoopDruidIndexerConfig.INDEX_MERGER_V9.persist(
index, interval, file, config.getIndexSpec(), progressIndicator, null
);
return HadoopDruidIndexerConfig.INDEX_MERGER_V9
.persist(index, interval, file, config.getIndexSpec(), progressIndicator, null);
}
protected File mergeQueryableIndex(
@ -600,9 +597,8 @@ public class IndexGeneratorJob implements Jobby
) throws IOException
{
boolean rollup = config.getSchema().getDataSchema().getGranularitySpec().isRollup();
return HadoopDruidIndexerConfig.INDEX_MERGER_V9.mergeQueryableIndex(
indexes, rollup, aggs, file, config.getIndexSpec(), progressIndicator, null
);
return HadoopDruidIndexerConfig.INDEX_MERGER_V9
.mergeQueryableIndex(indexes, rollup, aggs, file, config.getIndexSpec(), progressIndicator, null);
}
@Override
@ -624,9 +620,8 @@ public class IndexGeneratorJob implements Jobby
}
@Override
protected void reduce(
BytesWritable key, Iterable<BytesWritable> values, final Context context
) throws IOException, InterruptedException
protected void reduce(BytesWritable key, Iterable<BytesWritable> values, final Context context)
throws IOException, InterruptedException
{
SortableBytes keyBytes = SortableBytes.fromBytesWritable(key);
Bucket bucket = Bucket.fromGroupKey(keyBytes.getGroupKey()).lhs;
@ -775,9 +770,7 @@ public class IndexGeneratorJob implements Jobby
log.info("starting merge of intermediate persisted segments.");
long mergeStartTime = System.currentTimeMillis();
mergedBase = mergeQueryableIndex(
indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator
);
mergedBase = mergeQueryableIndex(indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator);
log.info(
"finished merge of intermediate persisted segments. time taken [%d] ms.",
(System.currentTimeMillis() - mergeStartTime)

View File

@ -412,7 +412,8 @@ public class JobHelper
final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration);
final AtomicLong size = new AtomicLong(0L);
final DataPusher zipPusher = (DataPusher) RetryProxy.create(
DataPusher.class, new DataPusher()
DataPusher.class,
new DataPusher()
{
@Override
public long push() throws IOException
@ -469,7 +470,8 @@ public class JobHelper
throws IOException
{
final DataPusher descriptorPusher = (DataPusher) RetryProxy.create(
DataPusher.class, new DataPusher()
DataPusher.class,
new DataPusher()
{
@Override
public long push() throws IOException
@ -694,7 +696,8 @@ public class JobHelper
}
final DataPusher zipPusher = (DataPusher) RetryProxy.create(
DataPusher.class, new DataPusher()
DataPusher.class,
new DataPusher()
{
@Override
public long push() throws IOException

View File

@ -115,10 +115,7 @@ public class SortableBytes
int b1Length = ByteBuffer.wrap(b1, s1 + 4, l1 - 4).getInt();
int b2Length = ByteBuffer.wrap(b2, s2 + 4, l2 - 4).getInt();
final int retVal = compareBytes(
b1, s1 + 8, b1Length,
b2, s2 + 8, b2Length
);
final int retVal = compareBytes(b1, s1 + 8, b1Length, b2, s2 + 8, b2Length);
return retVal;
}
@ -138,15 +135,16 @@ public class SortableBytes
int b1Length = ByteBuffer.wrap(b1, s1 + 4, l1 - 4).getInt();
int b2Length = ByteBuffer.wrap(b2, s2 + 4, l2 - 4).getInt();
int retVal = compareBytes(
b1, s1 + 8, b1Length,
b2, s2 + 8, b2Length
);
int retVal = compareBytes(b1, s1 + 8, b1Length, b2, s2 + 8, b2Length);
if (retVal == 0) {
retVal = compareBytes(
b1, s1 + 8 + b1Length, l1 - 8 - b1Length,
b2, s2 + 8 + b2Length, l2 - 8 - b2Length
b1,
s1 + 8 + b1Length,
l1 - 8 - b1Length,
b2,
s2 + 8 + b2Length,
l2 - 8 - b2Length
);
}

View File

@ -118,14 +118,9 @@ public class Utils
);
}
public static void storeStats(
JobContext job, Path path, Map<String, Object> stats
) throws IOException
public static void storeStats(JobContext job, Path path, Map<String, Object> stats) throws IOException
{
jsonMapper.writeValue(
makePathAndOutputStream(job, path, true),
stats
);
jsonMapper.writeValue(makePathAndOutputStream(job, path, true), stats);
}
public static String getFailureMessage(Job failedJob, ObjectMapper jsonMapper)

View File

@ -111,9 +111,7 @@ public class DatasourcePathSpec implements PathSpec
}
@Override
public Job addInputPaths(
HadoopDruidIndexerConfig config, Job job
) throws IOException
public Job addInputPaths(HadoopDruidIndexerConfig config, Job job) throws IOException
{
if (segments == null || segments.isEmpty()) {
if (ingestionSpec.isIgnoreWhenNoSegments()) {

View File

@ -43,9 +43,7 @@ public class MetadataStoreBasedUsedSegmentLister implements UsedSegmentLister
}
@Override
public List<DataSegment> getUsedSegmentsForIntervals(
String dataSource, List<Interval> intervals
)
public List<DataSegment> getUsedSegmentsForIntervals(String dataSource, List<Interval> intervals)
{
return indexerMetadataStorageCoordinator.getUsedSegmentsForIntervals(dataSource, intervals);
}

View File

@ -46,9 +46,7 @@ public class MultiplePathSpec implements PathSpec
}
@Override
public Job addInputPaths(
HadoopDruidIndexerConfig config, Job job
) throws IOException
public Job addInputPaths(HadoopDruidIndexerConfig config, Job job) throws IOException
{
for (PathSpec spec : children) {
spec.addInputPaths(config, job);

View File

@ -302,7 +302,8 @@ public class HadoopConverterJob
}
final List<DataSegment> returnList = ImmutableList.copyOf(
Lists.transform(
goodPaths, new Function<Path, DataSegment>()
goodPaths,
new Function<Path, DataSegment>()
{
@Nullable
@Override
@ -486,10 +487,7 @@ public class HadoopConverterJob
private static final String TMP_FILE_LOC_KEY = "org.apache.druid.indexer.updater.converter.reducer.tmpDir";
@Override
protected void map(
String key, String value,
final Context context
) throws IOException, InterruptedException
protected void map(String key, String value, final Context context) throws IOException, InterruptedException
{
final InputSplit split = context.getInputSplit();
if (!(split instanceof DatasourceInputSplit)) {
@ -623,7 +621,8 @@ public class HadoopConverterJob
throw new IOException("Bad config, missing segments");
}
return Lists.transform(
segments, new Function<DataSegment, InputSplit>()
segments,
new Function<DataSegment, InputSplit>()
{
@Nullable
@Override
@ -637,7 +636,8 @@ public class HadoopConverterJob
@Override
public RecordReader<String, String> createRecordReader(
final InputSplit inputSplit, final TaskAttemptContext taskAttemptContext
final InputSplit inputSplit,
final TaskAttemptContext taskAttemptContext
)
{
return new RecordReader<String, String>()

View File

@ -64,7 +64,9 @@ public class HadoopDruidConverterConfig
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("hadoop-converter", null, false, null, null, true, false)
binder,
Key.get(DruidNode.class, Self.class),
new DruidNode("hadoop-converter", null, false, null, null, true, false)
);
}
}

View File

@ -462,9 +462,7 @@ public class BatchDeltaIngestionTest
new LongSumAggregatorFactory("visited_sum", "visited_num"),
new HyperUniquesAggregatorFactory("unique_hosts", "host2")
},
new UniformGranularitySpec(
Granularities.DAY, Granularities.NONE, ImmutableList.of(INTERVAL_FULL)
),
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(INTERVAL_FULL)),
null,
MAPPER
),

View File

@ -237,7 +237,9 @@ public class DeterminePartitionsJobTest
),
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
new UniformGranularitySpec(
Granularities.DAY, Granularities.NONE, ImmutableList.of(Intervals.of(interval))
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(Intervals.of(interval))
),
null,
HadoopDruidIndexerConfig.JSON_MAPPER

View File

@ -81,7 +81,9 @@ public class IndexGeneratorCombinerTest
new HyperUniquesAggregatorFactory("unique_hosts", "host")
},
new UniformGranularitySpec(
Granularities.DAY, Granularities.NONE, ImmutableList.of(Intervals.of("2010/2011"))
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(Intervals.of("2010/2011"))
),
null,
HadoopDruidIndexerConfig.JSON_MAPPER

View File

@ -502,9 +502,7 @@ public class IndexGeneratorJobTest
Map.class
),
aggs,
new UniformGranularitySpec(
Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)
),
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)),
null,
mapper
),

View File

@ -92,9 +92,7 @@ public class JobHelperTest
Map.class
),
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
new UniformGranularitySpec(
Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)
),
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)),
null,
HadoopDruidIndexerConfig.JSON_MAPPER
),

View File

@ -48,18 +48,13 @@ public class SortableBytesTest
Assert.assertEquals(
-1,
WritableComparator.compareBytes(
thingie1Bytes, 0, thingie1Bytes.length,
thingie2Bytes, 0, thingie2Bytes.length
)
WritableComparator.compareBytes(thingie1Bytes, 0, thingie1Bytes.length, thingie2Bytes, 0, thingie2Bytes.length)
);
Assert.assertEquals(
0,
new SortableBytes.SortableBytesGroupingComparator().compare(
thingie1Bytes, 0, thingie1Bytes.length,
thingie2Bytes, 0, thingie2Bytes.length
)
new SortableBytes.SortableBytesGroupingComparator()
.compare(thingie1Bytes, 0, thingie1Bytes.length, thingie2Bytes, 0, thingie2Bytes.length)
);
SortableBytes reconThingie1 = SortableBytes.fromBytes(thingie1Bytes, 4, thingie1Bytes.length - 4);

View File

@ -151,35 +151,27 @@ public class DatasourceInputFormatTest
Path path4 = new Path(JobHelper.getURIFromSegment(segments2.get(0).getSegment()));
// dummy locations for test
locations = ImmutableList.of(
new LocatedFileStatus(
1000, false, 0, 0, 0, 0, null, null, null, null, path1,
new BlockLocation[]{
new BlockLocation(null, new String[]{"s1", "s2"}, 0, 600),
new BlockLocation(null, new String[]{"s2", "s3"}, 600, 400)
}
),
new LocatedFileStatus(
4000, false, 0, 0, 0, 0, null, null, null, null, path2,
new BlockLocation[]{
new BlockLocation(null, new String[]{"s1", "s2"}, 0, 1000),
new BlockLocation(null, new String[]{"s1", "s3"}, 1000, 1200),
new BlockLocation(null, new String[]{"s2", "s3"}, 2200, 1100),
new BlockLocation(null, new String[]{"s1", "s2"}, 3300, 700)
}
),
new LocatedFileStatus(
500, false, 0, 0, 0, 0, null, null, null, null, path3,
new BlockLocation[]{
new BlockLocation(null, new String[]{"s2", "s3"}, 0, 500)
}
),
new LocatedFileStatus(
500, false, 0, 0, 0, 0, null, null, null, null, path4,
new BlockLocation[]{
new BlockLocation(null, new String[]{"s2", "s3"}, 0, 500)
}
)
BlockLocation[] locations1 = {
new BlockLocation(null, new String[]{"s1", "s2"}, 0, 600),
new BlockLocation(null, new String[]{"s2", "s3"}, 600, 400)
};
BlockLocation[] locations2 = {
new BlockLocation(null, new String[]{"s1", "s2"}, 0, 1000),
new BlockLocation(null, new String[]{"s1", "s3"}, 1000, 1200),
new BlockLocation(null, new String[]{"s2", "s3"}, 2200, 1100),
new BlockLocation(null, new String[]{"s1", "s2"}, 3300, 700)
};
BlockLocation[] locations3 = {
new BlockLocation(null, new String[]{"s2", "s3"}, 0, 500)
};
BlockLocation[] locations4 = {
new BlockLocation(null, new String[]{"s2", "s3"}, 0, 500)
};
this.locations = ImmutableList.of(
new LocatedFileStatus(1000, false, 0, 0, 0, 0, null, null, null, null, path1, locations1),
new LocatedFileStatus(4000, false, 0, 0, 0, 0, null, null, null, null, path2, locations2),
new LocatedFileStatus(500, false, 0, 0, 0, 0, null, null, null, null, path3, locations3),
new LocatedFileStatus(500, false, 0, 0, 0, 0, null, null, null, null, path4, locations4)
);
config = populateConfiguration(new JobConf(), segments1, 0);

View File

@ -342,7 +342,9 @@ public class DatasourcePathSpecTest
new LongSumAggregatorFactory("visited_sum", "visited")
},
new UniformGranularitySpec(
Granularities.DAY, Granularities.NONE, ImmutableList.of(Intervals.of("2000/3000"))
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(Intervals.of("2000/3000"))
),
null,
HadoopDruidIndexerConfig.JSON_MAPPER

View File

@ -93,9 +93,7 @@ public class CheckPointDataSourceMetadataAction implements TaskAction<Boolean>
}
@Override
public Boolean perform(
Task task, TaskActionToolbox toolbox
)
public Boolean perform(Task task, TaskActionToolbox toolbox)
{
return toolbox.getSupervisorManager().checkPointDataSourceMetadata(
supervisorId,

View File

@ -59,9 +59,7 @@ public class ResetDataSourceMetadataAction implements TaskAction<Boolean>
}
@Override
public Boolean perform(
Task task, TaskActionToolbox toolbox
)
public Boolean perform(Task task, TaskActionToolbox toolbox)
{
return toolbox.getSupervisorManager().resetSupervisor(dataSource, resetMetadata);
}

View File

@ -63,9 +63,7 @@ public class SegmentMetadataUpdateAction implements TaskAction<Void>
}
@Override
public Void perform(
Task task, TaskActionToolbox toolbox
)
public Void perform(Task task, TaskActionToolbox toolbox)
{
TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments);

View File

@ -242,9 +242,8 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
RealtimeAppenderatorTuningConfig tuningConfig = spec.getTuningConfig()
.withBasePersistDirectory(toolbox.getPersistDir());
final FireDepartment fireDepartmentForMetrics = new FireDepartment(
dataSchema, new RealtimeIOConfig(null, null, null), null
);
final FireDepartment fireDepartmentForMetrics =
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null, null), null);
final TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(
this,

View File

@ -890,9 +890,8 @@ public class IndexTask extends AbstractTask implements ChatHandler
) throws IOException, InterruptedException
{
final GranularitySpec granularitySpec = dataSchema.getGranularitySpec();
final FireDepartment fireDepartmentForMetrics = new FireDepartment(
dataSchema, new RealtimeIOConfig(null, null, null), null
);
final FireDepartment fireDepartmentForMetrics =
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null, null), null);
buildSegmentsFireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
if (toolbox.getMonitorScheduler() != null) {

View File

@ -289,9 +289,8 @@ public class ParallelIndexSubTask extends AbstractTask
{
final DataSchema dataSchema = ingestionSchema.getDataSchema();
final GranularitySpec granularitySpec = dataSchema.getGranularitySpec();
final FireDepartment fireDepartmentForMetrics = new FireDepartment(
dataSchema, new RealtimeIOConfig(null, null, null), null
);
final FireDepartment fireDepartmentForMetrics =
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null, null), null);
final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
if (toolbox.getMonitorScheduler() != null) {

View File

@ -43,9 +43,7 @@ public class OverlordActionBasedUsedSegmentLister implements UsedSegmentLister
}
@Override
public List<DataSegment> getUsedSegmentsForIntervals(
String dataSource, List<Interval> intervals
) throws IOException
public List<DataSegment> getUsedSegmentsForIntervals(String dataSource, List<Interval> intervals) throws IOException
{
return toolbox
.getTaskActionClient()

View File

@ -222,7 +222,9 @@ public class HeapMemoryTaskStorage implements TaskStorage
@Override
public List<TaskInfo<Task, TaskStatus>> getRecentlyFinishedTaskInfo(
@Nullable Integer maxTaskStatuses, @Nullable Duration duration, @Nullable String datasource
@Nullable Integer maxTaskStatuses,
@Nullable Duration duration,
@Nullable String datasource
)
{
giant.lock();

View File

@ -780,7 +780,8 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
ImmutableMap.copyOf(
Maps.transformEntries(
Maps.filterEntries(
zkWorkers, new Predicate<Map.Entry<String, ZkWorker>>()
zkWorkers,
new Predicate<Map.Entry<String, ZkWorker>>()
{
@Override
public boolean apply(Map.Entry<String, ZkWorker> input)
@ -791,16 +792,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
}
}
),
new Maps.EntryTransformer<String, ZkWorker, ImmutableWorkerInfo>()
{
@Override
public ImmutableWorkerInfo transformEntry(
String key, ZkWorker value
)
{
return value.toImmutable();
}
}
(String key, ZkWorker value) -> value.toImmutable()
)
),
task

View File

@ -329,7 +329,8 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
ImmutableMap.copyOf(
Maps.transformEntries(
Maps.filterEntries(
workers, new Predicate<Map.Entry<String, WorkerHolder>>()
workers,
new Predicate<Map.Entry<String, WorkerHolder>>()
{
@Override
public boolean apply(Map.Entry<String, WorkerHolder> input)
@ -340,16 +341,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
}
}
),
new Maps.EntryTransformer<String, WorkerHolder, ImmutableWorkerInfo>()
{
@Override
public ImmutableWorkerInfo transformEntry(
String key, WorkerHolder value
)
{
return value.toImmutable();
}
}
(String key, WorkerHolder value) -> value.toImmutable()
)
),
task
@ -778,9 +770,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
}
@Override
public Collection<Worker> markWorkersLazy(
Predicate<ImmutableWorkerInfo> isLazyWorker, int maxWorkers
)
public Collection<Worker> markWorkersLazy(Predicate<ImmutableWorkerInfo> isLazyWorker, int maxWorkers)
{
synchronized (statusLock) {
Iterator<String> iterator = workers.keySet().iterator();

View File

@ -698,9 +698,8 @@ public class OverlordResource
final Interval theInterval = Intervals.of(interval.replace("_", "/"));
duration = theInterval.toDuration();
}
final List<TaskInfo<Task, TaskStatus>> taskInfoList = taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(
maxCompletedTasks, duration, dataSource
);
final List<TaskInfo<Task, TaskStatus>> taskInfoList =
taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(maxCompletedTasks, duration, dataSource);
final List<TaskStatusPlus> completedTasks = taskInfoList.stream()
.map(completeTaskTransformFunc::apply)
.collect(Collectors.toList());

View File

@ -74,7 +74,9 @@ public class JavaScriptWorkerSelectStrategy implements WorkerSelectStrategy
@Override
public ImmutableWorkerInfo findWorkerForTask(
WorkerTaskRunnerConfig config, ImmutableMap<String, ImmutableWorkerInfo> zkWorkers, Task task
WorkerTaskRunnerConfig config,
ImmutableMap<String, ImmutableWorkerInfo> zkWorkers,
Task task
)
{
fnSelector = fnSelector == null ? compileSelectorFunction() : fnSelector;

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