Make more package EverythingIsNonnullByDefault by default (#8198)

* Make more package EverythingIsNonnullByDefault by default

* Fixed additional voilations after pulling in master

* Change iterator to list.addAll

* Fix annotations
This commit is contained in:
Fokko Driesprong 2019-10-01 03:53:18 +03:00 committed by Gian Merlino
parent db65068c42
commit 82bfe86d0c
16 changed files with 116 additions and 44 deletions

View File

@ -19,11 +19,14 @@
package org.apache.druid.indexer;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
public interface TaskMetricsGetter
{
List<String> getKeys();
@Nullable
Map<String, Number> getTotalMetrics();
}

View File

@ -22,6 +22,7 @@ package org.apache.druid.java.util.common.collect;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
@ -64,9 +65,7 @@ public class Utils
list.add(null);
} else {
list = new ArrayList<>(elements.length);
for (T element : elements) {
list.add(element);
}
list.addAll(Arrays.asList(elements));
}
return list;
}

View File

@ -264,8 +264,11 @@ public class DetermineHashedPartitionsJob implements Jobby
public static class DetermineCardinalityMapper extends HadoopDruidIndexerMapper<LongWritable, BytesWritable>
{
private static HashFunction hashFunction = Hashing.murmur3_128();
@Nullable
private Granularity rollupGranularity = null;
@Nullable
private Map<Interval, HyperLogLogCollector> hyperLogLogs;
@Nullable
private HadoopDruidIndexerConfig config;
private boolean determineIntervals;
@ -349,6 +352,7 @@ public class DetermineHashedPartitionsJob implements Jobby
extends Reducer<LongWritable, BytesWritable, NullWritable, NullWritable>
{
private final List<Interval> intervals = new ArrayList<>();
@Nullable
protected HadoopDruidIndexerConfig config = null;
private boolean determineIntervals;
@ -429,8 +433,10 @@ public class DetermineHashedPartitionsJob implements Jobby
public static class DetermineHashedPartitionsPartitioner
extends Partitioner<LongWritable, BytesWritable> implements Configurable
{
@Nullable
private Configuration config;
private boolean determineIntervals;
@Nullable
private Map<LongWritable, Integer> reducerLookup;
@Override

View File

@ -325,6 +325,7 @@ public class DeterminePartitionsJob implements Jobby
public static class DeterminePartitionsGroupByMapper extends HadoopDruidIndexerMapper<BytesWritable, NullWritable>
{
@Nullable
private Granularity rollupGranularity = null;
@Override
@ -374,6 +375,7 @@ public class DeterminePartitionsJob implements Jobby
public static class DeterminePartitionsDimSelectionPostGroupByMapper
extends Mapper<BytesWritable, NullWritable, BytesWritable, Text>
{
@Nullable
private DeterminePartitionsDimSelectionMapperHelper helper;
@Override
@ -537,6 +539,7 @@ public class DeterminePartitionsJob implements Jobby
private abstract static class DeterminePartitionsDimSelectionBaseReducer
extends Reducer<BytesWritable, Text, BytesWritable, Text>
{
@Nullable
protected volatile HadoopDruidIndexerConfig config = null;
@Override
@ -905,6 +908,7 @@ public class DeterminePartitionsJob implements Jobby
private static class DimPartition
{
@Nullable
public ShardSpec shardSpec = null;
int cardinality = 0;
public long rows = 0;

View File

@ -30,7 +30,6 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Module;
@ -102,18 +101,13 @@ public class HadoopDruidIndexerConfig
INJECTOR = Initialization.makeInjectorWithModules(
GuiceInjectors.makeStartupInjector(),
ImmutableList.of(
new Module()
{
@Override
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
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);
}
(Module) binder -> {
JsonConfigProvider.bindInstance(
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);
},
new IndexingHadoopModule()
)

View File

@ -35,9 +35,13 @@ public class HadoopDruidIndexerJob implements Jobby
{
private static final Logger log = new Logger(HadoopDruidIndexerJob.class);
private final HadoopDruidIndexerConfig config;
@Nullable
private final MetadataStorageUpdaterJob metadataStorageUpdaterJob;
@Nullable
private IndexGeneratorJob indexJob;
@Nullable
private volatile List<DataSegment> publishedSegments = null;
@Nullable
private String hadoopJobIdFile;
@Inject

View File

@ -25,6 +25,8 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
import org.apache.druid.indexer.updater.MetadataStorageUpdaterJobSpec;
import org.apache.druid.segment.indexing.IOConfig;
import javax.annotation.Nullable;
import java.util.Map;
/**
@ -33,14 +35,16 @@ import java.util.Map;
public class HadoopIOConfig implements IOConfig
{
private final Map<String, Object> pathSpec;
@Nullable
private final MetadataStorageUpdaterJobSpec metadataUpdateSpec;
@Nullable
private final String segmentOutputPath;
@JsonCreator
public HadoopIOConfig(
final @JsonProperty("inputSpec") Map<String, Object> pathSpec,
final @JsonProperty("metadataUpdateSpec") MetadataStorageUpdaterJobSpec metadataUpdateSpec,
final @JsonProperty("segmentOutputPath") String segmentOutputPath
final @JsonProperty("metadataUpdateSpec") @Nullable MetadataStorageUpdaterJobSpec metadataUpdateSpec,
final @JsonProperty("segmentOutputPath") @Nullable String segmentOutputPath
)
{
this.pathSpec = pathSpec;
@ -54,12 +58,14 @@ public class HadoopIOConfig implements IOConfig
return pathSpec;
}
@Nullable
@JsonProperty("metadataUpdateSpec")
public MetadataStorageUpdaterJobSpec getMetadataUpdateSpec()
{
return metadataUpdateSpec;
}
@Nullable
@JsonProperty
public String getSegmentOutputPath()
{

View File

@ -34,6 +34,8 @@ import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.PartitionChunk;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
@ -56,8 +58,8 @@ public class HadoopIngestionSpec extends IngestionSpec<HadoopIOConfig, HadoopTun
public HadoopIngestionSpec(
@JsonProperty("dataSchema") DataSchema dataSchema,
@JsonProperty("ioConfig") HadoopIOConfig ioConfig,
@JsonProperty("tuningConfig") HadoopTuningConfig tuningConfig,
@JsonProperty("uniqueId") String uniqueId
@JsonProperty("tuningConfig") @Nullable HadoopTuningConfig tuningConfig,
@JsonProperty("uniqueId") @Nullable String uniqueId
)
{
super(dataSchema, ioConfig, tuningConfig);

View File

@ -76,7 +76,7 @@ public class HadoopTuningConfig implements TuningConfig
null
);
}
@Nullable
private final String workingPath;
private final String version;
private final DimensionBasedPartitionsSpec partitionsSpec;
@ -86,9 +86,9 @@ public class HadoopTuningConfig implements TuningConfig
private final int rowFlushBoundary;
private final long maxBytesInMemory;
private final boolean leaveIntermediate;
private final Boolean cleanupOnFailure;
private final boolean cleanupOnFailure;
private final boolean overwriteFiles;
private final Boolean ignoreInvalidRows;
private final boolean ignoreInvalidRows;
private final Map<String, String> jobProperties;
private final boolean combineText;
private final boolean useCombiner;
@ -102,29 +102,29 @@ public class HadoopTuningConfig implements TuningConfig
@JsonCreator
public HadoopTuningConfig(
final @JsonProperty("workingPath") String workingPath,
final @JsonProperty("version") String version,
final @JsonProperty("partitionsSpec") DimensionBasedPartitionsSpec partitionsSpec,
final @JsonProperty("shardSpecs") Map<Long, List<HadoopyShardSpec>> shardSpecs,
final @JsonProperty("indexSpec") IndexSpec indexSpec,
final @JsonProperty("workingPath") @Nullable String workingPath,
final @JsonProperty("version") @Nullable String version,
final @JsonProperty("partitionsSpec") @Nullable DimensionBasedPartitionsSpec partitionsSpec,
final @JsonProperty("shardSpecs") @Nullable Map<Long, List<HadoopyShardSpec>> shardSpecs,
final @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec,
final @JsonProperty("indexSpecForIntermediatePersists") @Nullable IndexSpec indexSpecForIntermediatePersists,
final @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory,
final @JsonProperty("maxBytesInMemory") Long maxBytesInMemory,
final @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory,
final @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory,
final @JsonProperty("leaveIntermediate") boolean leaveIntermediate,
final @JsonProperty("cleanupOnFailure") Boolean cleanupOnFailure,
final @JsonProperty("cleanupOnFailure") @Nullable Boolean cleanupOnFailure,
final @JsonProperty("overwriteFiles") boolean overwriteFiles,
final @Deprecated @JsonProperty("ignoreInvalidRows") Boolean ignoreInvalidRows,
final @JsonProperty("jobProperties") Map<String, String> jobProperties,
final @Deprecated @JsonProperty("ignoreInvalidRows") @Nullable Boolean ignoreInvalidRows,
final @JsonProperty("jobProperties") @Nullable Map<String, String> jobProperties,
final @JsonProperty("combineText") boolean combineText,
final @JsonProperty("useCombiner") Boolean useCombiner,
final @JsonProperty("useCombiner") @Nullable Boolean useCombiner,
// See https://github.com/apache/incubator-druid/pull/1922
final @JsonProperty("rowFlushBoundary") Integer maxRowsInMemoryCOMPAT,
final @JsonProperty("rowFlushBoundary") @Nullable Integer maxRowsInMemoryCOMPAT,
// This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12.
final @JsonProperty("buildV9Directly") Boolean buildV9Directly,
final @JsonProperty("numBackgroundPersistThreads") Integer numBackgroundPersistThreads,
final @JsonProperty("numBackgroundPersistThreads") @Nullable Integer numBackgroundPersistThreads,
final @JsonProperty("forceExtendableShardSpecs") boolean forceExtendableShardSpecs,
final @JsonProperty("useExplicitVersion") boolean useExplicitVersion,
final @JsonProperty("allowedHadoopPrefix") List<String> allowedHadoopPrefix,
final @JsonProperty("allowedHadoopPrefix") @Nullable List<String> allowedHadoopPrefix,
final @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions,
final @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions,
final @JsonProperty("useYarnRMJobStatusFallback") @Nullable Boolean useYarnRMJobStatusFallback
@ -150,7 +150,7 @@ public class HadoopTuningConfig implements TuningConfig
? ImmutableMap.of()
: ImmutableMap.copyOf(jobProperties));
this.combineText = combineText;
this.useCombiner = useCombiner == null ? DEFAULT_USE_COMBINER : useCombiner.booleanValue();
this.useCombiner = useCombiner == null ? DEFAULT_USE_COMBINER : useCombiner;
this.numBackgroundPersistThreads = numBackgroundPersistThreads == null
? DEFAULT_NUM_BACKGROUND_PERSIST_THREADS
: numBackgroundPersistThreads;
@ -174,6 +174,7 @@ public class HadoopTuningConfig implements TuningConfig
this.useYarnRMJobStatusFallback = useYarnRMJobStatusFallback == null ? true : useYarnRMJobStatusFallback;
}
@Nullable
@JsonProperty
public String getWorkingPath()
{

View File

@ -32,14 +32,13 @@ public class HadoopWorkingDirCleaner
{
private static final Logger log = new Logger(HadoopWorkingDirCleaner.class);
public static String runTask(String[] args) throws Exception
public static void runTask(String[] args) throws Exception
{
String workingPath = args[0];
log.info("Deleting indexing hadoop working path [%s].", workingPath);
Path p = new Path(workingPath);
FileSystem fs = p.getFileSystem(new Configuration());
fs.delete(p, true);
return null;
try (FileSystem fs = p.getFileSystem(new Configuration())) {
fs.delete(p, true);
}
}
}

View File

@ -408,6 +408,7 @@ public class InputRowSerde
return result;
}
@Nullable
private static List<String> readStringArray(DataInput in) throws IOException
{
int count = WritableUtils.readVInt(in);
@ -493,6 +494,7 @@ public class InputRowSerde
}
}
@Nullable
private static String getType(String metric, AggregatorFactory[] aggs, int i)
{
if (aggs[i].getName().equals(metric)) {

View File

@ -49,6 +49,8 @@ import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Progressable;
import javax.annotation.Nullable;
import java.io.BufferedOutputStream;
import java.io.File;
import java.io.FileInputStream;
@ -705,7 +707,7 @@ public class JobHelper
final Configuration configuration,
final File outDir,
final Progressable progressable,
final RetryPolicy retryPolicy
@Nullable final RetryPolicy retryPolicy
) throws IOException
{
final RetryPolicy effectiveRetryPolicy;

View File

@ -39,6 +39,8 @@ import org.apache.hadoop.util.ReflectionUtils;
import org.eclipse.jetty.client.HttpClient;
import org.eclipse.jetty.client.api.ContentResponse;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
@ -131,6 +133,7 @@ public class Utils
JSON_MAPPER.writeValue(makePathAndOutputStream(job, path, true), stats);
}
@Nullable
public static String getFailureMessage(Job failedJob, ObjectMapper jsonMapper)
{
try {

View File

@ -0,0 +1,23 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
@EverythingIsNonnullByDefault
package org.apache.druid.indexer;
import org.apache.druid.annotations.EverythingIsNonnullByDefault;

View File

@ -600,6 +600,7 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
return KEYS;
}
@Nullable
@Override
public Map<String, Number> getTotalMetrics()
{

View File

@ -0,0 +1,23 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
@EverythingIsNonnullByDefault
package org.apache.druid.indexing;
import org.apache.druid.annotations.EverythingIsNonnullByDefault;