From 782863ed0f00fe199599b6bafc587afd929e0f14 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Wed, 29 May 2019 20:20:45 +0200 Subject: [PATCH] Fix some problems reported by PVS-Studio (#7738) * Fix some problems reported by PVS-Studio * Address comments --- .gitignore | 1 + .../druid/benchmark/query/SelectBenchmark.java | 3 +-- .../druid/common/config/Log4jShutdown.java | 11 +++-------- .../segment/loading/DataSegmentKiller.java | 10 ++++++++++ .../druid/storage/azure/AzureTaskLogs.java | 6 +++--- .../firehose/kafka/KafkaSimpleConsumer.java | 4 +--- .../segment/MapVirtualColumnSelectTest.java | 3 +-- ...asicAuthenticatorMetadataStorageUpdater.java | 3 ++- ...orBasicAuthorizerMetadataStorageUpdater.java | 2 +- .../storage/google/GoogleDataSegmentKiller.java | 2 +- .../druid/storage/google/GoogleUtils.java | 9 --------- .../google/GoogleDataSegmentKillerTest.java | 3 ++- .../histogram/ApproximateHistogram.java | 2 +- .../data/input/orc/OrcStructConverter.java | 1 - .../avro/ParquetAvroHadoopInputRowParser.java | 2 +- .../parquet/simple/ParquetGroupConverter.java | 2 +- .../input/protobuf/ProtoTestEventWrapper.java | 3 +-- .../druid/storage/s3/S3DataSegmentKiller.java | 7 +------ .../org/apache/druid/storage/s3/S3Utils.java | 17 ----------------- .../hll/HyperLogLogCollectorBenchmark.java | 2 +- .../indexing/overlord/RemoteTaskRunner.java | 4 ++-- .../overlord/hrtr/HttpRemoteTaskRunnerTest.java | 15 +++++++++++++-- .../clients/CoordinatorResourceTestClient.java | 5 ----- .../query/select/SelectQueryQueryToolChest.java | 7 ++----- .../aggregation/AggregationTestHelper.java | 6 ++---- .../druid/query/search/SearchBinaryFnTest.java | 8 ++++++-- .../select/MultiSegmentSelectQueryTest.java | 3 +-- .../select/SelectQueryQueryToolChestTest.java | 3 +-- .../query/select/SelectQueryRunnerTest.java | 3 +-- .../org/apache/druid/segment/TestHelper.java | 16 ++++++---------- .../druid/client/selector/ServerSelector.java | 2 +- .../server/http/HostAndPortWithScheme.java | 9 +++++++-- .../client/CachingClusteredClientTest.java | 6 ++---- .../client/CachingClusteredClientTestUtils.java | 3 +-- .../client/cache/ByteCountingLRUMapTest.java | 8 ++++---- .../CuratorDruidCoordinatorTest.java | 5 +---- .../coordinator/DruidCoordinatorTest.java | 5 +---- .../coordinator/HttpLoadQueuePeonTest.java | 5 +---- .../server/coordinator/LoadQueuePeonTest.java | 10 ++-------- .../server/coordinator/LoadQueuePeonTester.java | 5 +---- .../coordinator/TestDruidCoordinatorConfig.java | 5 +---- .../DruidCoordinatorSegmentKillerTest.java | 5 +---- .../security/SecurityResourceFilterTest.java | 2 +- .../druid/sql/calcite/util/CalciteTests.java | 3 +-- 44 files changed, 91 insertions(+), 145 deletions(-) diff --git a/.gitignore b/.gitignore index 40e0adb4bcb..b3d45cf6e36 100644 --- a/.gitignore +++ b/.gitignore @@ -9,6 +9,7 @@ target .classpath .idea .project +.PVS-Studio .settings/ *.log *.DS_Store diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java index 28bbb1ff4a8..522e006d53d 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java @@ -233,8 +233,7 @@ public class SelectBenchmark factory = new SelectQueryRunnerFactory( new SelectQueryQueryToolChest( JSON_MAPPER, - QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator(), - selectConfigSupplier + QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator() ), new SelectQueryEngine(), QueryBenchmarkUtil.NOOP_QUERYWATCHER diff --git a/core/src/main/java/org/apache/druid/common/config/Log4jShutdown.java b/core/src/main/java/org/apache/druid/common/config/Log4jShutdown.java index cdfad64a5f6..918970c015a 100644 --- a/core/src/main/java/org/apache/druid/common/config/Log4jShutdown.java +++ b/core/src/main/java/org/apache/druid/common/config/Log4jShutdown.java @@ -158,7 +158,9 @@ public class Log4jShutdown implements ShutdownCallbackRegistry, LifeCycle private synchronized boolean compareAndSet(State expected, State transition) { if (current == expected) { - return transition(transition); + current = transition; + notifyAll(); + return true; } return false; } @@ -189,13 +191,6 @@ public class Log4jShutdown implements ShutdownCallbackRegistry, LifeCycle return current; } - private synchronized boolean transition(State transition) - { - current = transition; - notifyAll(); - return true; - } - private synchronized State get() { return current; diff --git a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentKiller.java b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentKiller.java index 20af38890a9..36c561269f0 100644 --- a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentKiller.java +++ b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentKiller.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.loading; import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; @@ -30,6 +31,15 @@ public interface DataSegmentKiller { Logger log = new Logger(DataSegmentKiller.class); + static String descriptorPath(String path) + { + int lastPathSeparatorIndex = path.lastIndexOf('/'); + if (lastPathSeparatorIndex == -1) { + throw new IAE("Invalid path: [%s], should contain '/'", path); + } + return path.substring(0, lastPathSeparatorIndex) + "/descriptor.json"; + } + /** * Removes segment files (index and metadata) from deep storage. * @param segment the segment to kill diff --git a/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureTaskLogs.java b/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureTaskLogs.java index d75d2008394..0196febea5b 100644 --- a/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureTaskLogs.java +++ b/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureTaskLogs.java @@ -53,7 +53,7 @@ public class AzureTaskLogs implements TaskLogs { final String taskKey = getTaskLogKey(taskid); log.info("Pushing task log %s to: %s", logFile, taskKey); - pushTaskFile(taskid, logFile, taskKey); + pushTaskFile(logFile, taskKey); } @Override @@ -61,10 +61,10 @@ public class AzureTaskLogs implements TaskLogs { final String taskKey = getTaskReportsKey(taskid); log.info("Pushing task reports %s to: %s", reportFile, taskKey); - pushTaskFile(taskid, reportFile, taskKey); + pushTaskFile(reportFile, taskKey); } - private void pushTaskFile(final String taskId, final File logFile, String taskKey) + private void pushTaskFile(final File logFile, String taskKey) { try { AzureUtils.retryAzureOperation( diff --git a/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/org/apache/druid/firehose/kafka/KafkaSimpleConsumer.java b/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/org/apache/druid/firehose/kafka/KafkaSimpleConsumer.java index 25fc8de15da..ce3028ff113 100644 --- a/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/org/apache/druid/firehose/kafka/KafkaSimpleConsumer.java +++ b/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/org/apache/druid/firehose/kafka/KafkaSimpleConsumer.java @@ -60,8 +60,6 @@ import java.util.concurrent.TimeUnit; public class KafkaSimpleConsumer { - public static final List EMPTY_MSGS = new ArrayList<>(); - private static final Logger log = new Logger(KafkaSimpleConsumer.class); private final List allBrokers; @@ -274,7 +272,7 @@ public class KafkaSimpleConsumer } } - return response != null ? filterAndDecode(response.messageSet(topic, partitionId), offset) : EMPTY_MSGS; + return filterAndDecode(response.messageSet(topic, partitionId), offset); } private void stopConsumer() diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java index f248bec0d3b..4dd24c4b32e 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java @@ -70,8 +70,7 @@ public class MapVirtualColumnSelectTest SelectQueryRunnerFactory factory = new SelectQueryRunnerFactory( new SelectQueryQueryToolChest( new DefaultObjectMapper(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), - selectConfigSupplier + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), new SelectQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java index 941b4a38a0e..ab6bb8ed292 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java @@ -91,7 +91,8 @@ public class CoordinatorBasicAuthenticatorMetadataStorageUpdater implements Basi BasicAuthCommonCacheConfig commonCacheConfig, @Smile ObjectMapper objectMapper, BasicAuthenticatorCacheNotifier cacheNotifier, - ConfigManager configManager // ConfigManager creates the db table we need, set a dependency here + ConfigManager configManager // -V6022 (unused parameter): ConfigManager creates the db table we need, + // set a dependency here ) { this.exec = Execs.scheduledSingleThreaded("CoordinatorBasicAuthenticatorMetadataStorageUpdater-Exec--%d"); diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java index 5c80ec7d9ba..e2b5849aad3 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java @@ -107,7 +107,7 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdater implements BasicAu BasicAuthCommonCacheConfig commonCacheConfig, @Smile ObjectMapper objectMapper, BasicAuthorizerCacheNotifier cacheNotifier, - ConfigManager configManager // ConfigManager creates the db table we need, set a dependency here + ConfigManager configManager // -V6022: ConfigManager creates the db table we need, set a dependency here ) { this.exec = Execs.scheduledSingleThreaded("CoordinatorBasicAuthorizerMetadataStorageUpdater-Exec--%d"); diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentKiller.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentKiller.java index be169012668..27cb989c1ad 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentKiller.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentKiller.java @@ -52,7 +52,7 @@ public class GoogleDataSegmentKiller implements DataSegmentKiller Map loadSpec = segment.getLoadSpec(); final String bucket = MapUtils.getString(loadSpec, "bucket"); final String indexPath = MapUtils.getString(loadSpec, "path"); - final String descriptorPath = indexPath.substring(0, indexPath.lastIndexOf('/')) + "/descriptor.json"; + final String descriptorPath = DataSegmentKiller.descriptorPath(indexPath); try { deleteIfPresent(bucket, indexPath); diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java index 6f23f9ca997..c3d3e4acb90 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleUtils.java @@ -25,15 +25,6 @@ import java.io.IOException; public class GoogleUtils { - public static String toFilename(String path) - { - return path.substring(path.lastIndexOf('/') + 1); // characters after last '/' - } - - public static String indexZipForSegmentPath(String path) - { - return path.substring(0, path.lastIndexOf('/')) + "/index.zip"; - } public static boolean isRetryable(Throwable t) { diff --git a/extensions-core/google-extensions/src/test/java/org/apache/druid/storage/google/GoogleDataSegmentKillerTest.java b/extensions-core/google-extensions/src/test/java/org/apache/druid/storage/google/GoogleDataSegmentKillerTest.java index 6ab2a1a4bb4..7b0812b37d3 100644 --- a/extensions-core/google-extensions/src/test/java/org/apache/druid/storage/google/GoogleDataSegmentKillerTest.java +++ b/extensions-core/google-extensions/src/test/java/org/apache/druid/storage/google/GoogleDataSegmentKillerTest.java @@ -24,6 +24,7 @@ import com.google.api.client.googleapis.testing.json.GoogleJsonResponseException import com.google.api.client.json.jackson2.JacksonFactory; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -40,7 +41,7 @@ public class GoogleDataSegmentKillerTest extends EasyMockSupport { private static final String bucket = "bucket"; private static final String indexPath = "test/2015-04-12T00:00:00.000Z_2015-04-13T00:00:00.000Z/1/0/index.zip"; - private static final String descriptorPath = indexPath.substring(0, indexPath.lastIndexOf('/')) + "/descriptor.json"; + private static final String descriptorPath = DataSegmentKiller.descriptorPath(indexPath); private static final DataSegment dataSegment = new DataSegment( "test", diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogram.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogram.java index 0135c8de18d..eeac3a53af6 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogram.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogram.java @@ -419,7 +419,7 @@ public class ApproximateHistogram // use unused slot to shift array left or right and make space for the new bin to insert if (insertAt < unusedIndex) { shiftRight(insertAt, unusedIndex); - } else if (insertAt >= unusedIndex) { + } else { shiftLeft(unusedIndex, insertAt - 1); insertAt--; } diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java index 20fbf069720..d27cc315927 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java @@ -166,7 +166,6 @@ public class OrcStructConverter fieldIndexCache.put(fields.get(i), i); } } - WritableComparable wc = struct.getFieldValue(fieldName); int fieldIndex = fieldIndexCache.getOrDefault(fieldName, -1); diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java index a297b94e5a1..1658d171758 100755 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java @@ -66,7 +66,7 @@ public class ParquetAvroHadoopInputRowParser implements InputRowParser loadSpec = segment.getLoadSpec(); String s3Bucket = MapUtils.getString(loadSpec, "bucket"); String s3Path = MapUtils.getString(loadSpec, "key"); - String s3DescriptorPath = descriptorPathForSegmentPath(s3Path); + String s3DescriptorPath = DataSegmentKiller.descriptorPath(s3Path); if (s3Client.doesObjectExist(s3Bucket, s3Path)) { log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); @@ -68,11 +68,6 @@ public class S3DataSegmentKiller implements DataSegmentKiller } } - private static String descriptorPathForSegmentPath(String s3Path) - { - return s3Path.substring(0, s3Path.lastIndexOf('/')) + "/descriptor.json"; - } - @Override public void killAll() { diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java index 1b6defd23d9..58ecdced699 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java @@ -172,23 +172,6 @@ public class S3Utils ) + "/index.zip"; } - static String indexZipForSegmentPath(String s3Path) - { - return s3Path.substring(0, s3Path.lastIndexOf('/')) + "/index.zip"; - } - - static String toFilename(String key) - { - return toFilename(key, ""); - } - - static String toFilename(String key, final String suffix) - { - String filename = key.substring(key.lastIndexOf('/') + 1); // characters after last '/' - filename = filename.substring(0, filename.length() - suffix.length()); // remove the suffix from the end - return filename; - } - static AccessControlList grantFullControlToBucketOwner(ServerSideEncryptingAmazonS3 s3Client, String bucket) { final AccessControlList acl = s3Client.getBucketAcl(bucket); diff --git a/hll/src/test/java/org/apache/druid/hll/HyperLogLogCollectorBenchmark.java b/hll/src/test/java/org/apache/druid/hll/HyperLogLogCollectorBenchmark.java index 26ef06f2c57..62446a6eaac 100644 --- a/hll/src/test/java/org/apache/druid/hll/HyperLogLogCollectorBenchmark.java +++ b/hll/src/test/java/org/apache/druid/hll/HyperLogLogCollectorBenchmark.java @@ -154,7 +154,7 @@ public class HyperLogLogCollectorBenchmark extends SimpleBenchmark final ByteBuffer buf = allocateEmptyHLLBuffer(targetIsDirect, alignTarget, 0); for (int k = 0; k < reps; ++k) { - for (int i = 0; i < count; ++i) { + for (int i = 0; i < count; ++i) { //-V6017: The 'k' counter is not used the nested loop because it's just reps. final int pos = positions[i]; final int size = sizes[i]; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java index fbe64f2a953..927f03efbfe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java @@ -724,8 +724,8 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer return; } final RemoteTaskRunnerWorkItem removed = completeTasks.remove(taskId); - final Worker worker = removed.getWorker(); - if (removed == null || worker == null) { + final Worker worker; + if (removed == null || (worker = removed.getWorker()) == null) { log.makeAlert("WTF?! Asked to cleanup nonexistent task") .addData("taskId", taskId) .emit(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java index b62a536ad1f..5eddb2e9e5b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java @@ -46,6 +46,7 @@ import org.apache.druid.indexing.overlord.config.HttpRemoteTaskRunnerConfig; import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; import org.apache.druid.indexing.worker.TaskAnnouncement; import org.apache.druid.indexing.worker.Worker; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.http.client.HttpClient; @@ -1285,8 +1286,18 @@ public class HttpRemoteTaskRunnerTest { return new WorkerHolder(smileMapper, httpClient, config, workersSyncExec, listener, worker) { - private final String workerHost = worker.getHost().substring(0, worker.getHost().indexOf(':')); - private final int workerPort = Integer.parseInt(worker.getHost().substring(worker.getHost().indexOf(':') + 1)); + private final String workerHost; + private final int workerPort; + + { + String hostAndPort = worker.getHost(); + int colonIndex = hostAndPort.indexOf(':'); + if (colonIndex == -1) { + throw new IAE("Invalid host and port: [%s]", colonIndex); + } + workerHost = hostAndPort.substring(0, colonIndex); + workerPort = Integer.parseInt(hostAndPort.substring(colonIndex + 1)); + } @Override public void start() diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java index babb9e3ead3..678169a640a 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java @@ -241,11 +241,6 @@ public class CoordinatorResourceTestClient ); } - Map results = jsonMapper.readValue( - response.getContent(), - new TypeReference>(){} - ); - StatusResponseHolder response2 = httpClient.go( new Request(HttpMethod.POST, new URL(url)).setContent( "application/json", diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java index 45ebb95a52d..9a5fb5a79a9 100644 --- a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; @@ -82,18 +81,16 @@ public class SelectQueryQueryToolChest extends QueryToolChest configSupplier + IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator ) { - this(jsonMapper, intervalChunkingQueryRunnerDecorator, configSupplier, DefaultSelectQueryMetricsFactory.instance()); + this(jsonMapper, intervalChunkingQueryRunnerDecorator, DefaultSelectQueryMetricsFactory.instance()); } @Inject public SelectQueryQueryToolChest( ObjectMapper jsonMapper, IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator, - Supplier configSupplier, SelectQueryMetricsFactory queryMetricsFactory ) { diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java index 3bc3ded2201..83b25e0d968 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java @@ -194,15 +194,13 @@ public class AggregationTestHelper implements Closeable SelectQueryQueryToolChest toolchest = new SelectQueryQueryToolChest( TestHelper.makeJsonMapper(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), - configSupplier + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ); SelectQueryRunnerFactory factory = new SelectQueryRunnerFactory( new SelectQueryQueryToolChest( TestHelper.makeJsonMapper(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), - configSupplier + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), new SelectQueryEngine( ), diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchBinaryFnTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchBinaryFnTest.java index ea566c46fe2..a48fc66506e 100644 --- a/processing/src/test/java/org/apache/druid/query/search/SearchBinaryFnTest.java +++ b/processing/src/test/java/org/apache/druid/query/search/SearchBinaryFnTest.java @@ -21,6 +21,7 @@ package org.apache.druid.query.search; import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.Result; import org.apache.druid.query.ordering.StringComparators; @@ -305,8 +306,11 @@ public class SearchBinaryFnTest { List result = new ArrayList<>(); for (String hit : hits) { - int index = hit.indexOf(':'); - result.add(new SearchHit(hit.substring(0, index), hit.substring(index + 1))); + int colonIndex = hit.indexOf(':'); + if (colonIndex == -1) { + throw new IAE("Invalid hit: [%s]", hit); + } + result.add(new SearchHit(hit.substring(0, colonIndex), hit.substring(colonIndex + 1))); } Collections.sort(result, comparator); return result; diff --git a/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java b/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java index 1b7eacc449f..d338b9e4a40 100644 --- a/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java @@ -75,8 +75,7 @@ public class MultiSegmentSelectQueryTest private static final SelectQueryQueryToolChest toolChest = new SelectQueryQueryToolChest( new DefaultObjectMapper(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), - configSupplier + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ); private static final QueryRunnerFactory factory = new SelectQueryRunnerFactory( diff --git a/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java index 46f12daad0f..2f8cfd6fbab 100644 --- a/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java @@ -38,8 +38,7 @@ public class SelectQueryQueryToolChestTest private static final SelectQueryQueryToolChest toolChest = new SelectQueryQueryToolChest( new DefaultObjectMapper(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), - configSupplier + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ); @Test diff --git a/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java index 3034315c598..55db5ef4e0d 100644 --- a/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java @@ -129,8 +129,7 @@ public class SelectQueryRunnerTest private static final SelectQueryQueryToolChest toolChest = new SelectQueryQueryToolChest( new DefaultObjectMapper(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), - configSupplier + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ); @Parameterized.Parameters(name = "{0}:descending={1}") diff --git a/processing/src/test/java/org/apache/druid/segment/TestHelper.java b/processing/src/test/java/org/apache/druid/segment/TestHelper.java index 19741e3bceb..3450e46407a 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/TestHelper.java @@ -316,16 +316,12 @@ public class TestHelper final Object actualValue = actualMap.get(key); if (expectedValue instanceof Float || expectedValue instanceof Double) { - if (expectedValue == null) { - Assert.assertNull(actualValue); - } else { - Assert.assertEquals( - StringUtils.format("%s: key[%s]", msg, key), - ((Number) expectedValue).doubleValue(), - ((Number) actualValue).doubleValue(), - Math.abs(((Number) expectedValue).doubleValue() * 1e-6) - ); - } + Assert.assertEquals( + StringUtils.format("%s: key[%s]", msg, key), + ((Number) expectedValue).doubleValue(), + ((Number) actualValue).doubleValue(), + Math.abs(((Number) expectedValue).doubleValue() * 1e-6) + ); } else { Assert.assertEquals( StringUtils.format("%s: key[%s]", msg, key), diff --git a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java index a485dbaa955..ac4fb84b477 100644 --- a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java +++ b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java @@ -127,7 +127,7 @@ public class ServerSelector implements DiscoverySelector .map(server -> server.getServer().getMetadata()) .forEach(candidates::add); - if (candidates.size() < numCandidates) { + if (candidates.size() < numCandidates) { //-V6007: false alarm due to a bug in PVS-Studio strategy.pick(realtimeServers, segment.get(), numCandidates - candidates.size()) .stream() .map(server -> server.getServer().getMetadata()) diff --git a/server/src/main/java/org/apache/druid/server/http/HostAndPortWithScheme.java b/server/src/main/java/org/apache/druid/server/http/HostAndPortWithScheme.java index 428a51d1401..0c584aab8d0 100644 --- a/server/src/main/java/org/apache/druid/server/http/HostAndPortWithScheme.java +++ b/server/src/main/java/org/apache/druid/server/http/HostAndPortWithScheme.java @@ -21,6 +21,7 @@ package org.apache.druid.server.http; import com.google.common.base.Preconditions; import com.google.common.net.HostAndPort; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; public class HostAndPortWithScheme @@ -42,9 +43,13 @@ public class HostAndPortWithScheme public static HostAndPortWithScheme fromString(String hostPortMaybeSchemeString) { if (hostPortMaybeSchemeString.startsWith("http")) { + int colonIndex = hostPortMaybeSchemeString.indexOf(':'); + if (colonIndex == -1) { + throw new IAE("Invalid host with scheme string: [%s]", hostPortMaybeSchemeString); + } return HostAndPortWithScheme.fromString( - hostPortMaybeSchemeString.substring(0, hostPortMaybeSchemeString.indexOf(':')), - hostPortMaybeSchemeString.substring(hostPortMaybeSchemeString.indexOf(':') + 1) + hostPortMaybeSchemeString.substring(0, colonIndex), + hostPortMaybeSchemeString.substring(colonIndex + 1) ); } return HostAndPortWithScheme.fromString("http", hostPortMaybeSchemeString); diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 59367c65716..de24926a3ab 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -1315,8 +1315,7 @@ public class CachingClusteredClientTest getDefaultQueryRunner(), new SelectQueryQueryToolChest( JSON_MAPPER, - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), - SELECT_CONFIG_SUPPLIER + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); HashMap context = new HashMap(); @@ -1393,8 +1392,7 @@ public class CachingClusteredClientTest getDefaultQueryRunner(), new SelectQueryQueryToolChest( JSON_MAPPER, - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), - SELECT_CONFIG_SUPPLIER + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); HashMap context = new HashMap(); diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java index d0571c79c8f..646031db7f9 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java @@ -92,8 +92,7 @@ public final class CachingClusteredClientTestUtils SelectQuery.class, new SelectQueryQueryToolChest( objectMapper, - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), - selectConfigSupplier + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ) .put( diff --git a/server/src/test/java/org/apache/druid/client/cache/ByteCountingLRUMapTest.java b/server/src/test/java/org/apache/druid/client/cache/ByteCountingLRUMapTest.java index de8285e70b5..e5c62db4370 100644 --- a/server/src/test/java/org/apache/druid/client/cache/ByteCountingLRUMapTest.java +++ b/server/src/test/java/org/apache/druid/client/cache/ByteCountingLRUMapTest.java @@ -92,10 +92,10 @@ public class ByteCountingLRUMapTest final ByteBuffer k = ByteBuffer.allocate(1); assertMapValues(0, 0, 0); - map.put(k, new byte[1]); - map.put(k, new byte[2]); - map.put(k, new byte[5]); - map.put(k, new byte[3]); + map.put(k, new byte[1]); //-V6033: suppress "An item with the same key has already been added" + map.put(k, new byte[2]); //-V6033 + map.put(k, new byte[5]); //-V6033 + map.put(k, new byte[3]); //-V6033 assertMapValues(1, 4, 0); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java index 3b2223514f5..53d6d34e7d3 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java @@ -160,10 +160,7 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase null, 10, null, - false, - false, - new Duration("PT0s"), - Duration.millis(10) + new Duration("PT0s") ); sourceLoadQueueChildrenCache = new PathChildrenCache( curator, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 793bd287b95..5ff12750495 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -137,10 +137,7 @@ public class DruidCoordinatorTest extends CuratorTestBase null, 10, null, - false, - false, - new Duration("PT0s"), - Duration.millis(10) + new Duration("PT0s") ); pathChildrenCache = new PathChildrenCache( curator, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java index 894472a2cbc..f45348d5ef6 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java @@ -82,10 +82,7 @@ public class HttpLoadQueuePeonTest null, 10, null, - false, - false, - Duration.ZERO, - Duration.millis(10) + Duration.ZERO ) { @Override diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java index 8d8271dab1d..3da088800a8 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java @@ -97,10 +97,7 @@ public class LoadQueuePeonTest extends CuratorTestBase null, 10, null, - false, - false, - Duration.millis(0), - Duration.millis(10) + Duration.millis(0) ) ); @@ -295,10 +292,7 @@ public class LoadQueuePeonTest extends CuratorTestBase null, 10, null, - false, - false, - new Duration("PT1s"), - Duration.millis(10) + new Duration("PT1s") ) ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java index c979671ff84..d71e903bfcc 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java @@ -46,10 +46,7 @@ public class LoadQueuePeonTester extends CuratorLoadQueuePeon null, 10, null, - false, - false, - new Duration("PT1s"), - Duration.millis(10) + new Duration("PT1s") ) ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java index e1b91354de7..03d8650493b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java @@ -44,10 +44,7 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig Duration coordinatorKillDurationToRetain, int coordinatorKillMaxSegments, String consoleStatic, - boolean mergeSegments, - boolean convertSegments, - Duration getLoadQueuePeonRepeatDelay, - Duration CuratorCreateZkNodesRepeatDelay + Duration getLoadQueuePeonRepeatDelay ) { this.coordinatorStartDelay = coordinatorStartDelay; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java index 7ee58a2a8f1..0f0bc033cb9 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java @@ -111,10 +111,7 @@ public class DruidCoordinatorSegmentKillerTest Duration.parse("PT86400S"), 1000, null, - false, - false, - Duration.ZERO, - Duration.millis(10) + Duration.ZERO ) ); diff --git a/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java b/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java index c3de4c883fb..4a17bf36d83 100644 --- a/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java +++ b/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java @@ -119,8 +119,8 @@ public class SecurityResourceFilterTest extends ResourceFilterTestHelper Assert.fail(); } catch (ForbiddenException e) { + EasyMock.verify(req, request, authorizerMapper); throw e; } - EasyMock.verify(req, request, authorizerMapper); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 590205ea28f..cc4e935c311 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -516,8 +516,7 @@ public class CalciteTests new SelectQueryRunnerFactory( new SelectQueryQueryToolChest( TestHelper.makeJsonMapper(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), - SELECT_CONFIG_SUPPLIER + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), new SelectQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER