Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
14 changes: 12 additions & 2 deletions .idea/inspectionProfiles/Druid.xml

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Original file line number Diff line number Diff line change
Expand Up @@ -564,7 +564,7 @@ public void registerSegmentCallback(Executor exec, SegmentCallback callback)
}
}

private class SimpleQueryRunner implements QueryRunner<Object>
private static class SimpleQueryRunner implements QueryRunner<Object>
{
private final QueryRunnerFactoryConglomerate conglomerate;
private final QueryableIndexSegment segment;
Expand Down Expand Up @@ -597,7 +597,7 @@ public Sequence<Object> run(QueryPlus<Object> queryPlus, ResponseContext respons
}
}

private class SingleSegmentDruidServer extends QueryableDruidServer<SimpleQueryRunner>
private static class SingleSegmentDruidServer extends QueryableDruidServer<SimpleQueryRunner>
{
SingleSegmentDruidServer(DruidServer server, SimpleQueryRunner runner)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ public class ParametrizedUriExtractor implements UriExtractor
public ParametrizedUriExtractor(String uriPattern)
{
this.uriPattern = uriPattern;
Matcher keyMatcher = Pattern.compile("\\{([^\\}]+)\\}").matcher(uriPattern);
Matcher keyMatcher = Pattern.compile("\\{([^}]+)}").matcher(uriPattern);
params = new HashSet<>();
while (keyMatcher.find()) {
params.add(keyMatcher.group(1));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1317,19 +1317,16 @@ public void testComplement8()
@Test
public void testComplement9()
{
final List<Integer> lengths = new ArrayList<Integer>();
lengths.addAll(
Arrays.asList(
35,
31,
32,
1,
0,
31 * 3,
1024,
ConciseSetUtils.MAX_ALLOWED_INTEGER
)
);
final List<Integer> lengths = new ArrayList<>(Arrays.asList(
35,
31,
32,
1,
0,
31 * 3,
1024,
ConciseSetUtils.MAX_ALLOWED_INTEGER
));
final Random random = new Random(701534702L);
for (int i = 0; i < 5; ++i) {
lengths.add(random.nextInt(ConciseSetUtils.MAX_ALLOWED_INTEGER + 1));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,9 +47,8 @@ public class MaterializedViewUtils
*/
public static Set<String> getRequiredFields(Query query)
{
Set<String> dimensions = new HashSet<>();
Set<String> dimsInFilter = null == query.getFilter() ? new HashSet<String>() : query.getFilter().getRequiredColumns();
dimensions.addAll(dimsInFilter);
Set<String> dimensions = new HashSet<>(dimsInFilter);

if (query instanceof TopNQuery) {
TopNQuery q = (TopNQuery) query;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,8 +129,7 @@ public MovingAverageQuery(
verifyOutputNames(this.dimensions, this.aggregatorSpecs, this.postAggregatorSpecs);

// build combined list of aggregators and averagers so that limit spec building is happy
List<AggregatorFactory> combinedAggregatorSpecs = new ArrayList<>();
combinedAggregatorSpecs.addAll(this.aggregatorSpecs);
List<AggregatorFactory> combinedAggregatorSpecs = new ArrayList<>(this.aggregatorSpecs);
for (AveragerFactory<?, ?> avg : this.averagerSpecs) {
combinedAggregatorSpecs.add(new AveragerFactoryWrapper(avg, ""));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -504,7 +504,7 @@ private void assertPostAggregatorSerde(PostAggregator agg) throws Exception
);
}

public static final String readFileFromClasspathAsString(String fileName) throws IOException
public static String readFileFromClasspathAsString(String fileName) throws IOException
{
return Files.asCharSource(
new File(SketchAggregationTest.class.getClassLoader().getResource(fileName).getFile()),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -313,7 +313,7 @@ public void testTopNQueryWithSketchConstant() throws Exception
Assert.assertEquals("product_2", value3.getDimensionValue("product"));
}

public static final String readFileFromClasspathAsString(String fileName) throws IOException
public static String readFileFromClasspathAsString(String fileName) throws IOException
{
return Files.asCharSource(
new File(SketchAggregationTest.class.getClassLoader().getResource(fileName).getFile()),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -253,7 +253,7 @@ private void assertPostAggregatorSerde(PostAggregator agg) throws Exception
);
}

public static final String readFileFromClasspathAsString(String fileName) throws IOException
public static String readFileFromClasspathAsString(String fileName) throws IOException
{
return Files.asCharSource(
new File(OldApiSketchAggregationTest.class.getClassLoader().getResource(fileName).getFile()),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,9 +65,7 @@ public PvaluefromZscorePostAggregator(
@Override
public Set<String> getDependentFields()
{
Set<String> dependentFields = new HashSet<>();

dependentFields.addAll(zScore.getDependentFields());
Set<String> dependentFields = new HashSet<>(zScore.getDependentFields());

return dependentFields;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ private static int sizes(byte[]... parts)
return size;
}

public static final Pair<Bucket, byte[]> fromGroupKey(byte[] keyBytes)
public static Pair<Bucket, byte[]> fromGroupKey(byte[] keyBytes)
{
ByteBuffer buf = ByteBuffer.wrap(keyBytes);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ public boolean run()
try {
if (!groupByJob.waitForCompletion(true)) {
log.error("Job failed: %s", groupByJob.getJobID());
failureCause = Utils.getFailureMessage(groupByJob, config.JSON_MAPPER);
failureCause = Utils.getFailureMessage(groupByJob, HadoopDruidIndexerConfig.JSON_MAPPER);
return false;
}
}
Expand All @@ -154,7 +154,7 @@ public boolean run()
if (!Utils.exists(groupByJob, fileSystem, intervalInfoPath)) {
throw new ISE("Path[%s] didn't exist!?", intervalInfoPath);
}
List<Interval> intervals = config.JSON_MAPPER.readValue(
List<Interval> intervals = HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
Utils.openInputStream(groupByJob, intervalInfoPath),
new TypeReference<List<Interval>>() {}
);
Expand All @@ -178,7 +178,7 @@ public boolean run()
fileSystem = partitionInfoPath.getFileSystem(groupByJob.getConfiguration());
}
if (Utils.exists(groupByJob, fileSystem, partitionInfoPath)) {
final Long numRows = config.JSON_MAPPER.readValue(
final Long numRows = HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
Utils.openInputStream(groupByJob, partitionInfoPath),
Long.class
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -281,7 +281,7 @@ public String getErrorMessage()
return null;
}

return Utils.getFailureMessage(job, config.JSON_MAPPER);
return Utils.getFailureMessage(job, HadoopDruidIndexerConfig.JSON_MAPPER);
}

private static IncrementalIndex makeIncrementalIndex(
Expand Down Expand Up @@ -823,16 +823,16 @@ public void doRun()
outputFS,
segmentTemplate,
JobHelper.INDEX_ZIP,
config.DATA_SEGMENT_PUSHER
HadoopDruidIndexerConfig.DATA_SEGMENT_PUSHER
),
JobHelper.makeTmpPath(
new Path(config.getSchema().getIOConfig().getSegmentOutputPath()),
outputFS,
segmentTemplate,
context.getTaskAttemptID(),
config.DATA_SEGMENT_PUSHER
HadoopDruidIndexerConfig.DATA_SEGMENT_PUSHER
),
config.DATA_SEGMENT_PUSHER
HadoopDruidIndexerConfig.DATA_SEGMENT_PUSHER
);

Path descriptorPath = config.makeDescriptorInfoPath(segment);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -289,7 +289,7 @@ public Double deserialize(ByteArrayDataInput in)
}
}

public static final SerializeResult toBytes(
public static SerializeResult toBytes(
final Map<String, IndexSerdeTypeHelper> typeHelperMap,
final InputRow row,
AggregatorFactory[] aggs
Expand Down Expand Up @@ -422,7 +422,7 @@ private static List<String> readStringArray(DataInput in) throws IOException
return values;
}

public static final InputRow fromBytes(
public static InputRow fromBytes(
final Map<String, IndexSerdeTypeHelper> typeHelperMap,
byte[] data,
AggregatorFactory[] aggs
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public class JobHelper
private static final int NUM_RETRIES = 8;
private static final int SECONDS_BETWEEN_RETRIES = 2;
private static final int DEFAULT_FS_BUFFER_SIZE = 1 << 18; // 256KB
private static final Pattern SNAPSHOT_JAR = Pattern.compile(".*\\-SNAPSHOT(-selfcontained)?\\.jar$");
private static final Pattern SNAPSHOT_JAR = Pattern.compile(".*-SNAPSHOT(-selfcontained)?\\.jar$");

public static Path distributedClassPath(String path)
{
Expand All @@ -103,8 +103,8 @@ public static Path distributedClassPath(Path base)
*/
public static void authenticate(HadoopDruidIndexerConfig config)
{
String principal = config.HADOOP_KERBEROS_CONFIG.getPrincipal();
String keytab = config.HADOOP_KERBEROS_CONFIG.getKeytab();
String principal = HadoopDruidIndexerConfig.HADOOP_KERBEROS_CONFIG.getPrincipal();
String keytab = HadoopDruidIndexerConfig.HADOOP_KERBEROS_CONFIG.getKeytab();
if (!Strings.isNullOrEmpty(principal) && !Strings.isNullOrEmpty(keytab)) {
Configuration conf = new Configuration();
UserGroupInformation.setConfiguration(conf);
Expand Down Expand Up @@ -180,7 +180,7 @@ public static void setupClasspath(
}
}

public static final Predicate<Throwable> shouldRetryPredicate()
public static Predicate<Throwable> shouldRetryPredicate()
{
return new Predicate<Throwable>()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -141,11 +141,11 @@ public static String getFailureMessage(Job failedJob, ObjectMapper jsonMapper)
TaskCompletionEvent[] completionEvents = failedJob.getTaskCompletionEvents(0, 100);
for (TaskCompletionEvent tce : completionEvents) {
String[] taskDiags = failedJob.getTaskDiagnostics(tce.getTaskAttemptId());
String combinedTaskDiags = "";
StringBuilder combinedTaskDiags = new StringBuilder();
for (String taskDiag : taskDiags) {
combinedTaskDiags += taskDiag;
combinedTaskDiags.append(taskDiag);
}
taskDiagsMap.put(tce.getTaskAttemptId().toString(), combinedTaskDiags);
taskDiagsMap.put(tce.getTaskAttemptId().toString(), combinedTaskDiags.toString());
}
return jsonMapper.writeValueAsString(taskDiagsMap);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,8 @@ public class InputRowSerdeTest
private List<String> dims;
private Map<String, Object> event;

{
static {
NullHandling.initializeForTests();
new AggregatorsModule(); //registers ComplexMetric serde for hyperUnique
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ private void testSerde(String path, Class inputFormat) throws Exception
Assert.assertEquals(path, pathSpec.getPaths());
}

public static final PathSpec readWriteRead(String jsonStr, ObjectMapper jsonMapper) throws Exception
public static PathSpec readWriteRead(String jsonStr, ObjectMapper jsonMapper) throws Exception
{
return jsonMapper.readValue(
jsonMapper.writeValueAsString(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ public class ExecutorLifecycleConfig
private File statusFile = null;

@JsonProperty
@Pattern(regexp = "\\{stdin\\}")
@Pattern(regexp = "\\{stdin}")
private String parentStreamName = "stdin";
@JsonProperty
private boolean parentStreamDefined = true;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -514,10 +514,9 @@ public void testCompactThenAppend() throws Exception
.interval(Intervals.of("2014-01-01/2014-01-02"))
.build();

final Set<DataSegment> expectedSegments = new HashSet<>();
final Pair<TaskStatus, List<DataSegment>> compactionResult = runTask(compactionTask);
Assert.assertTrue(compactionResult.lhs.isSuccess());
expectedSegments.addAll(compactionResult.rhs);
final Set<DataSegment> expectedSegments = new HashSet<>(compactionResult.rhs);

final Pair<TaskStatus, List<DataSegment>> appendResult = runAppendTask();
Assert.assertTrue(appendResult.lhs.isSuccess());
Expand Down
Loading