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

View File

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

View File

@ -50,7 +50,8 @@
<suppress checks="Indentation" files="[\\/]target[\\/]generated-test-sources[\\/]" /> <suppress checks="Indentation" files="[\\/]target[\\/]generated-test-sources[\\/]" />
<suppress checks="Indentation" files="ProtoTestEventWrapper.java" /> <suppress checks="Indentation" files="ProtoTestEventWrapper.java" />
<suppress checks="Regex" files="ProtoTestEventWrapper.java" />
<suppress checks="OneStatementPerLine" files="[\\/]target[\\/]generated-test-sources[\\/]" /> <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. --> <!-- 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 <property name="message" value="ObjectColumnSelector, LongColumnSelector, FloatColumnSelector
and DoubleColumnSelector must not be used in an instanceof statement, see Javadoc of those interfaces."/> and DoubleColumnSelector must not be used in an instanceof statement, see Javadoc of those interfaces."/>
</module> </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>
</module> </module>

View File

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

View File

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

View File

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

View File

@ -91,12 +91,8 @@ public class DruidSecondaryModule implements Module
mapper.setInjectableValues(new GuiceInjectableValues(injector)); mapper.setInjectableValues(new GuiceInjectableValues(injector));
mapper.setAnnotationIntrospectors( mapper.setAnnotationIntrospectors(
new AnnotationIntrospectorPair( new AnnotationIntrospectorPair(guiceIntrospector, mapper.getSerializationConfig().getAnnotationIntrospector()),
guiceIntrospector, mapper.getSerializationConfig().getAnnotationIntrospector() new AnnotationIntrospectorPair(guiceIntrospector, mapper.getDeserializationConfig().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 org.apache.druid.java.util.common.StringUtils;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.lang.annotation.Annotation;
import java.lang.reflect.ParameterizedType; import java.lang.reflect.ParameterizedType;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
@ -122,13 +123,10 @@ public class PolyBind
final TypeLiteral<T> interfaceType = interfaceKey.getTypeLiteral(); final TypeLiteral<T> interfaceType = interfaceKey.getTypeLiteral();
if (interfaceKey.getAnnotation() != null) { if (interfaceKey.getAnnotation() != null) {
return MapBinder.newMapBinder( return MapBinder.newMapBinder(binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotation());
binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotation()
);
} else if (interfaceKey.getAnnotationType() != null) { } else if (interfaceKey.getAnnotationType() != null) {
return MapBinder.newMapBinder( Class<? extends Annotation> annotationType = interfaceKey.getAnnotationType();
binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotationType() return MapBinder.newMapBinder(binder, TypeLiteral.get(String.class), interfaceType, annotationType);
);
} else { } else {
return MapBinder.newMapBinder(binder, TypeLiteral.get(String.class), interfaceType); return MapBinder.newMapBinder(binder, TypeLiteral.get(String.class), interfaceType);
} }
@ -178,7 +176,7 @@ public class PolyBind
if (key.getAnnotation() != null) { if (key.getAnnotation() != null) {
implsMap = (Map<String, Provider<T>>) injector.getInstance(Key.get(mapType, key.getAnnotation())); implsMap = (Map<String, Provider<T>>) injector.getInstance(Key.get(mapType, key.getAnnotation()));
} else if (key.getAnnotationType() != null) { } 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 { } else {
implsMap = (Map<String, Provider<T>>) injector.getInstance(Key.get(mapType)); implsMap = (Map<String, Provider<T>>) injector.getInstance(Key.get(mapType));
} }

View File

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

View File

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

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.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.JsonSerializable; import com.fasterxml.jackson.databind.JsonSerializable;
import com.fasterxml.jackson.databind.SerializerProvider; import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.jsontype.TypeSerializer; import com.fasterxml.jackson.databind.jsontype.TypeSerializer;
@ -428,9 +427,8 @@ public class PeriodGranularity extends Granularity implements JsonSerializable
} }
@Override @Override
public void serialize( public void serialize(JsonGenerator jsonGenerator, SerializerProvider serializerProvider)
JsonGenerator jsonGenerator, SerializerProvider serializerProvider throws IOException
) throws IOException, JsonProcessingException
{ {
// Retain the same behavior as before #3850. // Retain the same behavior as before #3850.
// i.e. when Granularity class was an enum. // i.e. when Granularity class was an enum.
@ -448,8 +446,10 @@ public class PeriodGranularity extends Granularity implements JsonSerializable
@Override @Override
public void serializeWithType( public void serializeWithType(
JsonGenerator jsonGenerator, SerializerProvider serializerProvider, TypeSerializer typeSerializer JsonGenerator jsonGenerator,
) throws IOException, JsonProcessingException SerializerProvider serializerProvider,
TypeSerializer typeSerializer
) throws IOException
{ {
serialize(jsonGenerator, serializerProvider); serialize(jsonGenerator, serializerProvider);
} }

View File

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

View File

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

View File

@ -71,9 +71,7 @@ public class FullResponseHandler implements HttpResponseHandler<FullResponseHold
} }
@Override @Override
public void exceptionCaught( public void exceptionCaught(ClientResponse<FullResponseHolder> clientResponse, Throwable e)
ClientResponse<FullResponseHolder> clientResponse, Throwable e
)
{ {
// Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished // 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 @Override
public void exceptionCaught( public void exceptionCaught(ClientResponse<StatusResponseHolder> clientResponse, Throwable e)
ClientResponse<StatusResponseHolder> clientResponse, Throwable e
)
{ {
// Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished // 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) { if (du != null) {
final Map<String, Long> stats = diff.to( 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/size", du.getReadBytes())
.put("sys/disk/read/count", du.getReads()) .put("sys/disk/read/count", du.getReads())
.put("sys/disk/write/size", du.getWriteBytes()) .put("sys/disk/write/size", du.getWriteBytes())
@ -379,7 +380,8 @@ public class SysMonitor extends FeedDefiningMonitor
} }
if (netstat != null) { if (netstat != null) {
final Map<String, Long> stats = diff.to( 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/size", netstat.getRxBytes())
.put("sys/net/read/packets", netstat.getRxPackets()) .put("sys/net/read/packets", netstat.getRxPackets())
.put("sys/net/read/errors", netstat.getRxErrors()) .put("sys/net/read/errors", netstat.getRxErrors())
@ -434,7 +436,8 @@ public class SysMonitor extends FeedDefiningMonitor
final Cpu cpu = cpus[i]; final Cpu cpu = cpus[i];
final String name = Integer.toString(i); final String name = Integer.toString(i);
final Map<String, Long> stats = diff.to( 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("user", cpu.getUser()) // user = Δuser / Δtotal
.put("sys", cpu.getSys()) // sys = Δsys / Δtotal .put("sys", cpu.getSys()) // sys = Δsys / Δtotal
.put("nice", cpu.getNice()) // nice = Δnice / Δtotal .put("nice", cpu.getNice()) // nice = Δnice / Δtotal

View File

@ -61,7 +61,9 @@ public interface DataSegmentFinder
* @param segmentModifiedAt segment modified timestamp * @param segmentModifiedAt segment modified timestamp
*/ */
static void putInMapRetainingNewest( 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( timestampedSegments.merge(

View File

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

View File

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

View File

@ -65,7 +65,10 @@ public class JsonConfiguratorTest
@Override @Override
public <T> Set<ConstraintViolation<T>> validateValue( 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(); return ImmutableSet.of();

View File

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

View File

@ -246,7 +246,8 @@ public class FriendlyServersTest
final ListenableFuture<StatusResponseHolder> response2 = skepticalClient final ListenableFuture<StatusResponseHolder> response2 = skepticalClient
.go( .go(
new Request( 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) new StatusResponseHandler(StandardCharsets.UTF_8)
); );

View File

@ -122,12 +122,8 @@ public class EvalTest
Assert.assertFalse(evalLong("!9223372036854775807", bindings) > 0); Assert.assertFalse(evalLong("!9223372036854775807", bindings) > 0);
Assert.assertEquals(3037000499L, evalLong("cast(sqrt(9223372036854775807), 'long')", bindings)); Assert.assertEquals(3037000499L, evalLong("cast(sqrt(9223372036854775807), 'long')", bindings));
Assert.assertEquals( Assert.assertEquals(1L, evalLong("if(x == 9223372036854775807, 1, 0)", bindings));
1L, evalLong("if(x == 9223372036854775807, 1, 0)", bindings) Assert.assertEquals(0L, evalLong("if(x - 1 == 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(1271030400000L, evalLong("timestamp('2010-04-12')", bindings));
Assert.assertEquals(1270998000000L, evalLong("timestamp('2010-04-12T+09:00')", 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 public class NoopDataSegmentMover implements DataSegmentMover
{ {
@Override @Override
public DataSegment move( public DataSegment move(DataSegment segment, Map<String, Object> targetLoadSpec)
DataSegment segment, Map<String, Object> targetLoadSpec
)
{ {
return segment; 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]", log.info("Retrieving file from region[%s], container[%s] and path [%s]",
region, container, path region, container, path
); );
CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy( CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
cloudFilesApi, region, container);
return new CloudFilesByteSource(objectApi, path); return new CloudFilesByteSource(objectApi, path);
} }

View File

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

View File

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

View File

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

View File

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

View File

@ -196,9 +196,7 @@ public class KafkaSimpleConsumer
earliest ? kafka.api.OffsetRequest.EarliestTime() : kafka.api.OffsetRequest.LatestTime(), 1 earliest ? kafka.api.OffsetRequest.EarliestTime() : kafka.api.OffsetRequest.LatestTime(), 1
) )
); );
OffsetRequest request = new OffsetRequest( OffsetRequest request = new OffsetRequest(requestInfo, kafka.api.OffsetRequest.CurrentVersion(), clientId);
requestInfo, kafka.api.OffsetRequest.CurrentVersion(), clientId
);
OffsetResponse response = null; OffsetResponse response = null;
try { try {
response = consumer.getOffsetsBefore(request); response = consumer.getOffsetsBefore(request);
@ -323,7 +321,11 @@ public class KafkaSimpleConsumer
catch (Exception e) { catch (Exception e) {
ensureNotInterrupted(e); ensureNotInterrupted(e);
log.warn( 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 { finally {

View File

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

View File

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

View File

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

View File

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

View File

@ -90,7 +90,9 @@ public class InlineSchemaAvroBytesDecoder implements AvroBytesDecoder
catch (EOFException eof) { catch (EOFException eof) {
// waiting for avro v1.9.0 (#AVRO-813) // waiting for avro v1.9.0 (#AVRO-813)
throw new ParseException( 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) { catch (Exception e) {

View File

@ -122,7 +122,9 @@ public class InlineSchemasAvroBytesDecoder implements AvroBytesDecoder
catch (EOFException eof) { catch (EOFException eof) {
// waiting for avro v1.9.0 (#AVRO-813) // waiting for avro v1.9.0 (#AVRO-813)
throw new ParseException( 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) { catch (Exception e) {

View File

@ -84,7 +84,9 @@ public class SchemaRepoBasedAvroBytesDecoder<SUBJECT, ID> implements AvroBytesDe
catch (EOFException eof) { catch (EOFException eof) {
// waiting for avro v1.9.0 (#AVRO-813) // waiting for avro v1.9.0 (#AVRO-813)
throw new ParseException( 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) { 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 // 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(1, inputRow.getDimension("someIntValueMap").size());
assertEquals( assertEquals(
SOME_INT_VALUE_MAP_VALUE, new HashMap<CharSequence, Integer>( SOME_INT_VALUE_MAP_VALUE,
new HashMap<CharSequence, Integer>(
Maps.transformValues( Maps.transformValues(
Splitter.on(",") Splitter.on(",")
.withKeyValueSeparator("=") .withKeyValueSeparator("=")
@ -299,7 +300,8 @@ public class AvroStreamInputRowParserTest
) )
); );
assertEquals( assertEquals(
SOME_STRING_VALUE_MAP_VALUE, new HashMap<CharSequence, CharSequence>( SOME_STRING_VALUE_MAP_VALUE,
new HashMap<CharSequence, CharSequence>(
Splitter.on(",") Splitter.on(",")
.withKeyValueSeparator("=") .withKeyValueSeparator("=")
.split(inputRow.getDimension("someIntValueMap").get(0).replaceAll("[\\{\\} ]", "")) .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(SketchSetPostAggregator.class, THETA_SKETCH_SET_OP_POST_AGG),
new NamedType(SketchConstantPostAggregator.class, THETA_SKETCH_CONSTANT_POST_AGG) new NamedType(SketchConstantPostAggregator.class, THETA_SKETCH_CONSTANT_POST_AGG)
) )
.addSerializer( .addSerializer(SketchHolder.class, new SketchHolderJsonSerializer())
SketchHolder.class, new SketchHolderJsonSerializer()
)
); );
} }
} }

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -95,7 +95,10 @@ public class KafkaLookupExtractorFactoryTest
{ {
@Override @Override
public Object findInjectableValue( 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)) { if ("org.apache.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager".equals(valueId)) {

View File

@ -67,7 +67,11 @@ public class KafkaIndexTaskClient extends IndexTaskClient
try { try {
final FullResponseHolder response = submitRequestWithEmptyContent( 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); return isSuccess(response);
} }

View File

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

View File

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

View File

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

View File

@ -91,7 +91,10 @@ public class NamespaceLookupExtractorFactoryTest
{ {
@Override @Override
public Object findInjectableValue( 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)) { if (CacheScheduler.class.getName().equals(valueId)) {
@ -524,7 +527,9 @@ public class NamespaceLookupExtractorFactoryTest
public void configure(Binder binder) public void configure(Binder binder)
{ {
JsonConfigProvider.bindInstance( 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(); final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector();
mapper.setAnnotationIntrospectors( mapper.setAnnotationIntrospectors(
new AnnotationIntrospectorPair( new AnnotationIntrospectorPair(
guiceIntrospector, mapper.getSerializationConfig().getAnnotationIntrospector() guiceIntrospector,
mapper.getSerializationConfig().getAnnotationIntrospector()
), ),
new AnnotationIntrospectorPair( new AnnotationIntrospectorPair(
guiceIntrospector, mapper.getDeserializationConfig().getAnnotationIntrospector() guiceIntrospector,
mapper.getDeserializationConfig().getAnnotationIntrospector()
) )
); );
return mapper; return mapper;

View File

@ -49,7 +49,9 @@ public class OffHeapNamespaceExtractionCacheManagerTest
public void configure(Binder binder) public void configure(Binder binder)
{ {
JsonConfigProvider.bindInstance( 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) public void configure(Binder binder)
{ {
JsonConfigProvider.bindInstance( 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. * Constructs a new {@code CacheStats} instance.
*/ */
public LookupCacheStats( public LookupCacheStats(long hitCount, long missCount, long evictionCount)
long hitCount, long missCount,
long evictionCount
)
{ {
Preconditions.checkArgument(hitCount >= 0); Preconditions.checkArgument(hitCount >= 0);
Preconditions.checkArgument(missCount >= 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(); final Set<V> setOfValues = setOfValuesBuilder.build();
reverseCache.putAll(Maps.asMap( reverseCache.putAll(Maps.asMap(
setOfValues, new Function<V, List<K>>() setOfValues,
new Function<V, List<K>>()
{ {
@Override @Override
public List<K> apply(final V input) public List<K> apply(final V input)
@ -80,7 +81,8 @@ public class OffHeapPollingCache<K, V> implements PollingCache<K, V>
} }
}).keySet()); }).keySet());
} }
})); }
));
started.getAndSet(true); started.getAndSet(true);
} }
} }

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -412,7 +412,8 @@ public class JobHelper
final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration); final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration);
final AtomicLong size = new AtomicLong(0L); final AtomicLong size = new AtomicLong(0L);
final DataPusher zipPusher = (DataPusher) RetryProxy.create( final DataPusher zipPusher = (DataPusher) RetryProxy.create(
DataPusher.class, new DataPusher() DataPusher.class,
new DataPusher()
{ {
@Override @Override
public long push() throws IOException public long push() throws IOException
@ -469,7 +470,8 @@ public class JobHelper
throws IOException throws IOException
{ {
final DataPusher descriptorPusher = (DataPusher) RetryProxy.create( final DataPusher descriptorPusher = (DataPusher) RetryProxy.create(
DataPusher.class, new DataPusher() DataPusher.class,
new DataPusher()
{ {
@Override @Override
public long push() throws IOException public long push() throws IOException
@ -694,7 +696,8 @@ public class JobHelper
} }
final DataPusher zipPusher = (DataPusher) RetryProxy.create( final DataPusher zipPusher = (DataPusher) RetryProxy.create(
DataPusher.class, new DataPusher() DataPusher.class,
new DataPusher()
{ {
@Override @Override
public long push() throws IOException 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 b1Length = ByteBuffer.wrap(b1, s1 + 4, l1 - 4).getInt();
int b2Length = ByteBuffer.wrap(b2, s2 + 4, l2 - 4).getInt(); int b2Length = ByteBuffer.wrap(b2, s2 + 4, l2 - 4).getInt();
final int retVal = compareBytes( final int retVal = compareBytes(b1, s1 + 8, b1Length, b2, s2 + 8, b2Length);
b1, s1 + 8, b1Length,
b2, s2 + 8, b2Length
);
return retVal; return retVal;
} }
@ -138,15 +135,16 @@ public class SortableBytes
int b1Length = ByteBuffer.wrap(b1, s1 + 4, l1 - 4).getInt(); int b1Length = ByteBuffer.wrap(b1, s1 + 4, l1 - 4).getInt();
int b2Length = ByteBuffer.wrap(b2, s2 + 4, l2 - 4).getInt(); int b2Length = ByteBuffer.wrap(b2, s2 + 4, l2 - 4).getInt();
int retVal = compareBytes( int retVal = compareBytes(b1, s1 + 8, b1Length, b2, s2 + 8, b2Length);
b1, s1 + 8, b1Length,
b2, s2 + 8, b2Length
);
if (retVal == 0) { if (retVal == 0) {
retVal = compareBytes( retVal = compareBytes(
b1, s1 + 8 + b1Length, l1 - 8 - b1Length, b1,
b2, s2 + 8 + b2Length, l2 - 8 - b2Length 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( public static void storeStats(JobContext job, Path path, Map<String, Object> stats) throws IOException
JobContext job, Path path, Map<String, Object> stats
) throws IOException
{ {
jsonMapper.writeValue( jsonMapper.writeValue(makePathAndOutputStream(job, path, true), stats);
makePathAndOutputStream(job, path, true),
stats
);
} }
public static String getFailureMessage(Job failedJob, ObjectMapper jsonMapper) public static String getFailureMessage(Job failedJob, ObjectMapper jsonMapper)

View File

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

View File

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

View File

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

View File

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

View File

@ -64,7 +64,9 @@ public class HadoopDruidConverterConfig
public void configure(Binder binder) public void configure(Binder binder)
{ {
JsonConfigProvider.bindInstance( 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 LongSumAggregatorFactory("visited_sum", "visited_num"),
new HyperUniquesAggregatorFactory("unique_hosts", "host2") new HyperUniquesAggregatorFactory("unique_hosts", "host2")
}, },
new UniformGranularitySpec( new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(INTERVAL_FULL)),
Granularities.DAY, Granularities.NONE, ImmutableList.of(INTERVAL_FULL)
),
null, null,
MAPPER MAPPER
), ),

View File

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

View File

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

View File

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

View File

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

View File

@ -48,18 +48,13 @@ public class SortableBytesTest
Assert.assertEquals( Assert.assertEquals(
-1, -1,
WritableComparator.compareBytes( WritableComparator.compareBytes(thingie1Bytes, 0, thingie1Bytes.length, thingie2Bytes, 0, thingie2Bytes.length)
thingie1Bytes, 0, thingie1Bytes.length,
thingie2Bytes, 0, thingie2Bytes.length
)
); );
Assert.assertEquals( Assert.assertEquals(
0, 0,
new SortableBytes.SortableBytesGroupingComparator().compare( new SortableBytes.SortableBytesGroupingComparator()
thingie1Bytes, 0, thingie1Bytes.length, .compare(thingie1Bytes, 0, thingie1Bytes.length, thingie2Bytes, 0, thingie2Bytes.length)
thingie2Bytes, 0, thingie2Bytes.length
)
); );
SortableBytes reconThingie1 = SortableBytes.fromBytes(thingie1Bytes, 4, thingie1Bytes.length - 4); 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())); Path path4 = new Path(JobHelper.getURIFromSegment(segments2.get(0).getSegment()));
// dummy locations for test // dummy locations for test
locations = ImmutableList.of( BlockLocation[] locations1 = {
new LocatedFileStatus( new BlockLocation(null, new String[]{"s1", "s2"}, 0, 600),
1000, false, 0, 0, 0, 0, null, null, null, null, path1, new BlockLocation(null, new String[]{"s2", "s3"}, 600, 400)
new BlockLocation[]{ };
new BlockLocation(null, new String[]{"s1", "s2"}, 0, 600), BlockLocation[] locations2 = {
new BlockLocation(null, new String[]{"s2", "s3"}, 600, 400) 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 LocatedFileStatus( new BlockLocation(null, new String[]{"s1", "s2"}, 3300, 700)
4000, false, 0, 0, 0, 0, null, null, null, null, path2, };
new BlockLocation[]{ BlockLocation[] locations3 = {
new BlockLocation(null, new String[]{"s1", "s2"}, 0, 1000), new BlockLocation(null, new String[]{"s2", "s3"}, 0, 500)
new BlockLocation(null, new String[]{"s1", "s3"}, 1000, 1200), };
new BlockLocation(null, new String[]{"s2", "s3"}, 2200, 1100), BlockLocation[] locations4 = {
new BlockLocation(null, new String[]{"s1", "s2"}, 3300, 700) new BlockLocation(null, new String[]{"s2", "s3"}, 0, 500)
} };
), this.locations = ImmutableList.of(
new LocatedFileStatus( new LocatedFileStatus(1000, false, 0, 0, 0, 0, null, null, null, null, path1, locations1),
500, false, 0, 0, 0, 0, null, null, null, null, path3, new LocatedFileStatus(4000, false, 0, 0, 0, 0, null, null, null, null, path2, locations2),
new BlockLocation[]{ new LocatedFileStatus(500, false, 0, 0, 0, 0, null, null, null, null, path3, locations3),
new BlockLocation(null, new String[]{"s2", "s3"}, 0, 500) new LocatedFileStatus(500, false, 0, 0, 0, 0, null, null, null, null, path4, locations4)
}
),
new LocatedFileStatus(
500, false, 0, 0, 0, 0, null, null, null, null, path4,
new BlockLocation[]{
new BlockLocation(null, new String[]{"s2", "s3"}, 0, 500)
}
)
); );
config = populateConfiguration(new JobConf(), segments1, 0); config = populateConfiguration(new JobConf(), segments1, 0);

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -222,7 +222,9 @@ public class HeapMemoryTaskStorage implements TaskStorage
@Override @Override
public List<TaskInfo<Task, TaskStatus>> getRecentlyFinishedTaskInfo( 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(); giant.lock();

View File

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

View File

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

View File

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

View File

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

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