Fix some problems reported by PVS-Studio (#7738)

* Fix some problems reported by PVS-Studio

* Address comments
This commit is contained in:
Roman Leventov 2019-05-29 20:20:45 +02:00 committed by Gian Merlino
parent b051d6688d
commit 782863ed0f
44 changed files with 91 additions and 145 deletions

1
.gitignore vendored
View File

@ -9,6 +9,7 @@ target
.classpath
.idea
.project
.PVS-Studio
.settings/
*.log
*.DS_Store

View File

@ -233,8 +233,7 @@ public class SelectBenchmark
factory = new SelectQueryRunnerFactory(
new SelectQueryQueryToolChest(
JSON_MAPPER,
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator(),
selectConfigSupplier
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
),
new SelectQueryEngine(),
QueryBenchmarkUtil.NOOP_QUERYWATCHER

View File

@ -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;

View File

@ -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

View File

@ -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(

View File

@ -60,8 +60,6 @@ import java.util.concurrent.TimeUnit;
public class KafkaSimpleConsumer
{
public static final List<BytesMessageWithOffset> EMPTY_MSGS = new ArrayList<>();
private static final Logger log = new Logger(KafkaSimpleConsumer.class);
private final List<HostAndPort> 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()

View File

@ -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

View File

@ -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");

View File

@ -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");

View File

@ -52,7 +52,7 @@ public class GoogleDataSegmentKiller implements DataSegmentKiller
Map<String, Object> 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);

View File

@ -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)
{

View File

@ -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",

View File

@ -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--;
}

View File

@ -166,7 +166,6 @@ public class OrcStructConverter
fieldIndexCache.put(fields.get(i), i);
}
}
WritableComparable wc = struct.getFieldValue(fieldName);
int fieldIndex = fieldIndexCache.getOrDefault(fieldName, -1);

View File

@ -66,7 +66,7 @@ public class ParquetAvroHadoopInputRowParser implements InputRowParser<GenericRe
this.binaryAsString = binaryAsString == null ? false : binaryAsString;
final JSONPathSpec flattenSpec;
if (parseSpec != null && (parseSpec instanceof AvroParseSpec)) {
if (parseSpec instanceof AvroParseSpec) {
flattenSpec = ((AvroParseSpec) parseSpec).getFlattenSpec();
} else {
flattenSpec = JSONPathSpec.DEFAULT;

View File

@ -397,7 +397,7 @@ class ParquetGroupConverter
return bytes;
}
default:
throw new RE("Unknown primitive conversion: %s", ot.name());
throw new RE("Unknown primitive conversion: %s", pt.getPrimitiveTypeName());
}
}
}

View File

@ -710,8 +710,7 @@ public final class ProtoTestEventWrapper
}
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo other = (org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) obj;
boolean result = true;
result = result && (hasBar() == other.hasBar());
boolean result = hasBar() == other.hasBar();
if (hasBar()) {
result = result && getBar()
.equals(other.getBar());

View File

@ -50,7 +50,7 @@ public class S3DataSegmentKiller implements DataSegmentKiller
Map<String, Object> 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()
{

View File

@ -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);

View File

@ -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];

View File

@ -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();

View File

@ -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()

View File

@ -241,11 +241,6 @@ public class CoordinatorResourceTestClient
);
}
Map<String, Object> results = jsonMapper.readValue(
response.getContent(),
new TypeReference<Map<String, Object>>(){}
);
StatusResponseHolder response2 = httpClient.go(
new Request(HttpMethod.POST, new URL(url)).setContent(
"application/json",

View File

@ -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<Result<SelectResul
public SelectQueryQueryToolChest(
ObjectMapper jsonMapper,
IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator,
Supplier<SelectQueryConfig> configSupplier
IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator
)
{
this(jsonMapper, intervalChunkingQueryRunnerDecorator, configSupplier, DefaultSelectQueryMetricsFactory.instance());
this(jsonMapper, intervalChunkingQueryRunnerDecorator, DefaultSelectQueryMetricsFactory.instance());
}
@Inject
public SelectQueryQueryToolChest(
ObjectMapper jsonMapper,
IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator,
Supplier<SelectQueryConfig> configSupplier,
SelectQueryMetricsFactory queryMetricsFactory
)
{

View File

@ -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(
),

View File

@ -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<SearchHit> 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;

View File

@ -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(

View File

@ -38,8 +38,7 @@ public class SelectQueryQueryToolChestTest
private static final SelectQueryQueryToolChest toolChest = new SelectQueryQueryToolChest(
new DefaultObjectMapper(),
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
configSupplier
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
@Test

View File

@ -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}")

View File

@ -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),

View File

@ -127,7 +127,7 @@ public class ServerSelector implements DiscoverySelector<QueryableDruidServer>
.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())

View File

@ -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);

View File

@ -1315,8 +1315,7 @@ public class CachingClusteredClientTest
getDefaultQueryRunner(),
new SelectQueryQueryToolChest(
JSON_MAPPER,
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
SELECT_CONFIG_SUPPLIER
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
);
HashMap<String, Object> context = new HashMap<String, Object>();
@ -1393,8 +1392,7 @@ public class CachingClusteredClientTest
getDefaultQueryRunner(),
new SelectQueryQueryToolChest(
JSON_MAPPER,
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
SELECT_CONFIG_SUPPLIER
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
);
HashMap<String, Object> context = new HashMap<String, Object>();

View File

@ -92,8 +92,7 @@ public final class CachingClusteredClientTestUtils
SelectQuery.class,
new SelectQueryQueryToolChest(
objectMapper,
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
selectConfigSupplier
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
)
.put(

View File

@ -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);
}

View File

@ -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,

View File

@ -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,

View File

@ -82,10 +82,7 @@ public class HttpLoadQueuePeonTest
null,
10,
null,
false,
false,
Duration.ZERO,
Duration.millis(10)
Duration.ZERO
)
{
@Override

View File

@ -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")
)
);

View File

@ -46,10 +46,7 @@ public class LoadQueuePeonTester extends CuratorLoadQueuePeon
null,
10,
null,
false,
false,
new Duration("PT1s"),
Duration.millis(10)
new Duration("PT1s")
)
);
}

View File

@ -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;

View File

@ -111,10 +111,7 @@ public class DruidCoordinatorSegmentKillerTest
Duration.parse("PT86400S"),
1000,
null,
false,
false,
Duration.ZERO,
Duration.millis(10)
Duration.ZERO
)
);

View File

@ -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);
}
}

View File

@ -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