Replace NOT operator with explicit `false` check - part 8 (#68625)

Part 8.

We have an in-house rule to compare explicitly against `false` instead
of using the logical not operator (`!`). However, this hasn't
historically been enforced, meaning that there are many violations in
the source at present.

We now have a Checkstyle rule that can detect these cases, but before we
can turn it on, we need to fix the existing violations. This is being
done over a series of PRs, since there are a lot to fix.
This commit is contained in:
Rory Hunter 2021-02-08 15:20:34 +00:00 committed by GitHub
parent b29fe24391
commit 780f273067
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
122 changed files with 223 additions and 205 deletions

View File

@ -53,7 +53,7 @@ public final class MetricsCalculator {
metrics.add(new Metrics(operationAndMetrics.getKey(),
samples.stream().filter((r) -> r.isSuccess()).count(),
samples.stream().filter((r) -> !r.isSuccess()).count(),
samples.stream().filter((r) -> r.isSuccess() == false).count(),
// throughput calculation is based on the total (Wall clock) time it took to generate all samples
calculateThroughput(samples.size(), latestEnd - firstStart),
// convert ns -> ms without losing precision

View File

@ -80,6 +80,6 @@ public final class DefaultDetectorDescription {
}
private static boolean isNotNullOrEmpty(String arg) {
return !Strings.isNullOrEmpty(arg);
return Strings.isNullOrEmpty(arg) == false;
}
}

View File

@ -27,7 +27,7 @@ public class CompletableContext<T> {
if (t == null) {
listener.accept(v, null);
} else {
assert !(t instanceof Error) : "Cannot be error";
assert (t instanceof Error) == false: "Cannot be error";
listener.accept(v, (Exception) t);
}
};

View File

@ -117,7 +117,7 @@ public final class DissectParser {
}
this.maxMatches = matchPairs.size();
this.maxResults = Long.valueOf(matchPairs.stream()
.filter(dissectPair -> !dissectPair.getKey().skip()).map(KEY_NAME).distinct().count()).intValue();
.filter(dissectPair -> dissectPair.getKey().skip() == false).map(KEY_NAME).distinct().count()).intValue();
if (this.maxMatches == 0 || maxResults == 0) {
throw new DissectException.PatternParse(pattern, "Unable to find any keys or delimiters.");
}

View File

@ -112,7 +112,7 @@ public class DissectKeyTests extends ESTestCase {
public void testMultipleLeftModifiers() {
String keyName = randomAlphaOfLengthBetween(1, 10);
List<String> validModifiers = EnumSet.allOf(DissectKey.Modifier.class).stream()
.filter(m -> !m.equals(DissectKey.Modifier.NONE))
.filter(m -> m.equals(DissectKey.Modifier.NONE) == false)
.map(DissectKey.Modifier::toString)
.collect(Collectors.toList());
String modifier1 = randomFrom(validModifiers);

View File

@ -53,7 +53,7 @@ public class Geohash {
for(int i = 1; i <= PRECISION; i++) {
precisionToLatHeight[i] = precisionToLatHeight[i-1] / (even ? 8 : 4);
precisionToLonWidth[i] = precisionToLonWidth[i-1] / (even ? 4 : 8);
even = ! even;
even = even == false;
}
}

View File

@ -29,8 +29,8 @@ public enum SslClientAuthenticationMode {
public void configure(SSLParameters sslParameters) {
// nothing to do here
assert !sslParameters.getWantClientAuth();
assert !sslParameters.getNeedClientAuth();
assert sslParameters.getWantClientAuth() == false;
assert sslParameters.getNeedClientAuth() == false;
}
},
/**

View File

@ -92,6 +92,6 @@ public class FilterPathBasedFilter extends TokenFilter {
@Override
protected boolean _includeScalar() {
return !inclusive;
return inclusive == false;
}
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
import java.io.IOException;
import java.util.Objects;
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
@ -199,15 +200,17 @@ public class SnapshotIndexShardStatus extends BroadcastShardResponse implements
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SnapshotIndexShardStatus that = (SnapshotIndexShardStatus) o;
if (stage != that.stage) return false;
if (stats != null ? !stats.equals(that.stats) : that.stats != null) return false;
if (nodeId != null ? !nodeId.equals(that.nodeId) : that.nodeId != null) return false;
return failure != null ? failure.equals(that.failure) : that.failure == null;
return stage == that.stage
&& Objects.equals(stats, that.stats)
&& Objects.equals(nodeId, that.nodeId)
&& Objects.equals(failure, that.failure);
}
@Override

View File

@ -22,6 +22,7 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableMap;
@ -149,15 +150,17 @@ public class SnapshotIndexStatus implements Iterable<SnapshotIndexShardStatus>,
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SnapshotIndexStatus that = (SnapshotIndexStatus) o;
if (index != null ? !index.equals(that.index) : that.index != null) return false;
if (indexShards != null ? !indexShards.equals(that.indexShards) : that.indexShards != null) return false;
if (shardsStats != null ? !shardsStats.equals(that.shardsStats) : that.shardsStats != null) return false;
return stats != null ? stats.equals(that.stats) : that.stats == null;
return Objects.equals(index, that.index)
&& Objects.equals(indexShards, that.indexShards)
&& Objects.equals(shardsStats, that.shardsStats)
&& Objects.equals(stats, that.stats);
}
@Override

View File

@ -27,6 +27,7 @@ import org.elasticsearch.index.query.QueryBuilder;
import java.io.IOException;
import java.io.InputStream;
import java.util.Map;
import java.util.Objects;
/**
* Represents an alias, to be associated with an index
@ -289,9 +290,7 @@ public class Alias implements Writeable, ToXContentFragment {
Alias alias = (Alias) o;
if (name != null ? !name.equals(alias.name) : alias.name != null) return false;
return true;
return Objects.equals(name, alias.name);
}
@Override

View File

@ -474,7 +474,7 @@ public class AnalyzeAction extends ActionType<AnalyzeAction.Response> {
if (positionLength > 1) {
builder.field(POSITION_LENGTH, positionLength);
}
if (attributes != null && !attributes.isEmpty()) {
if (attributes != null && attributes.isEmpty() == false) {
Map<String, Object> sortedAttributes = new TreeMap<>(attributes);
for (Map.Entry<String, Object> entity : sortedAttributes.entrySet()) {
builder.field(entity.getKey(), entity.getValue());

View File

@ -107,7 +107,7 @@ public class CommonStatsFlags implements Writeable, Cloneable {
}
public boolean anySet() {
return !flags.isEmpty();
return flags.isEmpty() == false;
}
public Flag[] getFlags() {

View File

@ -49,7 +49,7 @@ public class GetIndexTemplatesRequest extends MasterNodeReadRequest<GetIndexTemp
validationException = addValidationError("names is null or empty", validationException);
} else {
for (String name : names) {
if (name == null || !Strings.hasText(name)) {
if (name == null || Strings.hasText(name) == false) {
validationException = addValidationError("name is missing", validationException);
}
}

View File

@ -101,7 +101,7 @@ public class ValidateQueryResponse extends BroadcastResponse {
@Override
protected void addCustomXContentFields(XContentBuilder builder, Params params) throws IOException {
builder.field(VALID_FIELD, isValid());
if (getQueryExplanation() != null && !getQueryExplanation().isEmpty()) {
if (getQueryExplanation() != null && getQueryExplanation().isEmpty() == false) {
builder.startArray(EXPLANATIONS_FIELD);
for (QueryExplanation explanation : getQueryExplanation()) {
builder.startObject();

View File

@ -408,14 +408,14 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques
}
private static String valueOrDefault(String value, String globalDefault) {
if (Strings.isNullOrEmpty(value) && !Strings.isNullOrEmpty(globalDefault)) {
if (Strings.isNullOrEmpty(value) && Strings.isNullOrEmpty(globalDefault) == false) {
return globalDefault;
}
return value;
}
private static Boolean valueOrDefault(Boolean value, Boolean globalDefault) {
if (Objects.isNull(value) && !Objects.isNull(globalDefault)) {
if (Objects.isNull(value) && Objects.isNull(globalDefault) == false) {
return globalDefault;
}
return value;

View File

@ -97,7 +97,7 @@ public class Retry {
finishHim();
} else {
if (canRetry(bulkItemResponses)) {
addResponses(bulkItemResponses, (r -> !r.isFailed()));
addResponses(bulkItemResponses, (r -> r.isFailed() == false));
retry(createBulkRequestForRetry(bulkItemResponses));
} else {
addResponses(bulkItemResponses, (r -> true));

View File

@ -88,7 +88,7 @@ public class TransportGetAction extends TransportSingleShardAction<GetRequest, G
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardId.id());
if (request.refresh() && !request.realtime()) {
if (request.refresh() && request.realtime() == false) {
indexShard.refresh("refresh_flag_get");
}

View File

@ -90,7 +90,7 @@ public class TransportShardMultiGetAction extends TransportSingleShardAction<Mul
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardId.id());
if (request.refresh() && !request.realtime()) {
if (request.refresh() && request.realtime() == false) {
indexShard.refresh("refresh_flag_mget");
}

View File

@ -64,7 +64,7 @@ public class GetPipelineResponse extends ActionResponse implements StatusToXCont
}
public boolean isFound() {
return !pipelines.isEmpty();
return pipelines.isEmpty() == false;
}
@Override

View File

@ -97,7 +97,7 @@ public final class SearchPhaseController {
}
assert !lEntry.fieldStatistics().containsKey(null);
assert lEntry.fieldStatistics().containsKey(null) == false;
final Object[] keys = lEntry.fieldStatistics().keys;
final Object[] values = lEntry.fieldStatistics().values;
for (int i = 0; i < keys.length; i++) {

View File

@ -32,7 +32,7 @@ public class TransportActions {
* If a failure is already present, should this failure override it or not for read operations.
*/
public static boolean isReadOverrideException(Exception e) {
return !isShardNotAvailableException(e);
return isShardNotAvailableException(e) == false;
}
}

View File

@ -352,7 +352,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
protected void onNodeFailure(DiscoveryNode node, int nodeIndex, Throwable t) {
String nodeId = node.getId();
if (logger.isDebugEnabled() && !(t instanceof NodeShouldNotConnectException)) {
if (logger.isDebugEnabled() && (t instanceof NodeShouldNotConnectException) == false) {
logger.debug(new ParameterizedMessage("failed to execute [{}] on node [{}]", actionName, nodeId), t);
}

View File

@ -128,7 +128,7 @@ public abstract class TransportMasterNodeAction<Request extends MasterNodeReques
retry(clusterState, blockException, newState -> {
try {
ClusterBlockException newException = checkBlock(request, newState);
return (newException == null || !newException.retryable());
return (newException == null || newException.retryable() == false);
} catch (Exception e) {
// accept state as block will be rechecked by doStart() and listener.onFailure() then called
logger.debug("exception occurred during cluster block checking, accepting state", e);

View File

@ -227,7 +227,7 @@ public abstract class TransportNodesAction<NodesRequest extends BaseNodesRequest
}
private void onFailure(int idx, String nodeId, Throwable t) {
if (logger.isDebugEnabled() && !(t instanceof NodeShouldNotConnectException)) {
if (logger.isDebugEnabled() && (t instanceof NodeShouldNotConnectException) == false) {
logger.debug(new ParameterizedMessage("failed to execute on node [{}]", nodeId), t);
}
responses.set(idx, new FailedNodeException(nodeId, "Failed node [" + nodeId + "]", t));

View File

@ -279,7 +279,7 @@ public abstract class TransportTasksAction<
}
private void onFailure(int idx, String nodeId, Throwable t) {
if (logger.isDebugEnabled() && !(t instanceof NodeShouldNotConnectException)) {
if (logger.isDebugEnabled() && (t instanceof NodeShouldNotConnectException) == false) {
logger.debug(new ParameterizedMessage("failed to execute on node [{}]", nodeId), t);
}

View File

@ -457,7 +457,7 @@ public class TermVectorsRequest extends SingleShardRequest<TermVectorsRequest> i
flagsEnum.add(flag);
} else if (set == false) {
flagsEnum.remove(flag);
assert (!flagsEnum.contains(flag));
assert flagsEnum.contains(flag) == false;
}
}

View File

@ -48,7 +48,7 @@ final class TermVectorsWriter {
boolean hasScores = termVectorsFilter != null;
for (String field : termVectorsByField) {
if ((selectedFields != null) && (!selectedFields.contains(field))) {
if (selectedFields != null && selectedFields.contains(field) == false) {
continue;
}
@ -85,7 +85,7 @@ final class TermVectorsWriter {
Term term = new Term(field, termBytesRef);
// with filtering we only keep the best terms
if (hasScores && !termVectorsFilter.hasScoreTerm(term)) {
if (hasScores && termVectorsFilter.hasScoreTerm(term) == false) {
continue;
}

View File

@ -114,7 +114,7 @@ public class UpdateHelper {
* {@code IndexRequest} to be executed on the primary and replicas.
*/
Result prepareUpsert(ShardId shardId, UpdateRequest request, final GetResult getResult, LongSupplier nowInMillis) {
if (request.upsertRequest() == null && !request.docAsUpsert()) {
if (request.upsertRequest() == null && request.docAsUpsert() == false) {
throw new DocumentMissingException(shardId, request.id());
}
IndexRequest indexRequest = request.docAsUpsert() ? request.doc() : request.upsertRequest();
@ -177,7 +177,7 @@ public class UpdateHelper {
final XContentType updateSourceContentType = sourceAndContent.v1();
final Map<String, Object> updatedSourceAsMap = sourceAndContent.v2();
final boolean noop = !XContentHelper.update(updatedSourceAsMap, currentRequest.sourceAsMap(), detectNoop);
final boolean noop = XContentHelper.update(updatedSourceAsMap, currentRequest.sourceAsMap(), detectNoop) == false;
// We can only actually turn the update into a noop if detectNoop is true to preserve backwards compatibility and to handle cases
// where users repopulating multi-fields or adding synonyms, etc.

View File

@ -42,7 +42,7 @@ public interface BootstrapCheck {
}
public boolean isFailure() {
return !isSuccess();
return isSuccess() == false;
}
public String getMessage() {

View File

@ -172,9 +172,9 @@ final class BootstrapChecks {
static boolean enforceLimits(final BoundTransportAddress boundTransportAddress, final String discoveryType) {
final Predicate<TransportAddress> isLoopbackAddress = t -> t.address().getAddress().isLoopbackAddress();
final boolean bound =
!(Arrays.stream(boundTransportAddress.boundAddresses()).allMatch(isLoopbackAddress) &&
isLoopbackAddress.test(boundTransportAddress.publishAddress()));
return bound && !"single-node".equals(discoveryType);
(Arrays.stream(boundTransportAddress.boundAddresses()).allMatch(isLoopbackAddress) &&
isLoopbackAddress.test(boundTransportAddress.publishAddress())) == false;
return bound && "single-node".equals(discoveryType) == false;
}
// the list of checks to execute
@ -578,7 +578,7 @@ final class BootstrapChecks {
@Override
boolean mightFork() {
final String onError = onError();
return onError != null && !onError.equals("");
return onError != null && onError.isEmpty() == false;
}
// visible for testing
@ -603,7 +603,7 @@ final class BootstrapChecks {
@Override
boolean mightFork() {
final String onOutOfMemoryError = onOutOfMemoryError();
return onOutOfMemoryError != null && !onOutOfMemoryError.equals("");
return onOutOfMemoryError != null && onOutOfMemoryError.isEmpty() == false;
}
// visible for testing

View File

@ -156,7 +156,7 @@ class Elasticsearch extends EnvironmentAwareCommand {
void init(final boolean daemonize, final Path pidFile, final boolean quiet, Environment initialEnv)
throws NodeValidationException, UserException {
try {
Bootstrap.init(!daemonize, pidFile, quiet, initialEnv);
Bootstrap.init(daemonize == false, pidFile, quiet, initialEnv);
} catch (BootstrapException | RuntimeException e) {
// format exceptions to the console in a special way
// to avoid 2MB stacktraces from guice, etc.

View File

@ -137,7 +137,7 @@ public interface ClusterStateTaskExecutor<T> {
}
public Exception getFailure() {
assert !isSuccess();
assert isSuccess() == false;
return failure;
}
}

View File

@ -242,7 +242,7 @@ public class CoordinationState {
boolean added = joinVotes.addJoinVote(join);
boolean prevElectionWon = electionWon;
electionWon = isElectionQuorum(joinVotes);
assert !prevElectionWon || electionWon : // we cannot go from won to not won
assert prevElectionWon == false || electionWon : // we cannot go from won to not won
"locaNode= " + localNode + ", join=" + join + ", joinVotes=" + joinVotes;
logger.debug("handleJoin: added join {} from [{}] for election, electionWon={} lastAcceptedTerm={} lastAcceptedVersion={}", join,
join.getSourceNode(), electionWon, lastAcceptedTerm, getLastAcceptedVersion());

View File

@ -200,7 +200,9 @@ public class IndexNameExpressionResolver {
// If only one index is specified then whether we fail a request if an index is missing depends on the allow_no_indices
// option. At some point we should change this, because there shouldn't be a reason why whether a single index
// or multiple indices are specified yield different behaviour.
final boolean failNoIndices = indexExpressions.length == 1 ? !options.allowNoIndices() : !options.ignoreUnavailable();
final boolean failNoIndices = indexExpressions.length == 1
? options.allowNoIndices() == false
: options.ignoreUnavailable() == false;
List<String> expressions = Arrays.asList(indexExpressions);
for (ExpressionResolver expressionResolver : expressionResolvers) {
expressions = expressionResolver.resolve(context, expressions);

View File

@ -75,7 +75,7 @@ public class SystemIndexMetadataUpgradeService implements ClusterStateListener {
for (ObjectObjectCursor<String, IndexMetadata> cursor : indexMetadataMap) {
if (cursor.value != lastIndexMetadataMap.get(cursor.key)) {
if (systemIndices.isSystemIndex(cursor.value.getIndex()) != cursor.value.isSystem()) {
updatedMetadata.add(IndexMetadata.builder(cursor.value).system(!cursor.value.isSystem()).build());
updatedMetadata.add(IndexMetadata.builder(cursor.value).system(cursor.value.isSystem() == false).build());
}
}
}

View File

@ -461,7 +461,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
}
public boolean hasChanges() {
return masterNodeChanged() || !removed.isEmpty() || !added.isEmpty();
return masterNodeChanged() || removed.isEmpty() == false || added.isEmpty() == false;
}
public boolean masterNodeChanged() {
@ -479,7 +479,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
}
public boolean removed() {
return !removed.isEmpty();
return removed.isEmpty() == false;
}
public List<DiscoveryNode> removedNodes() {
@ -487,7 +487,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
}
public boolean added() {
return !added.isEmpty();
return added.isEmpty() == false;
}
public List<DiscoveryNode> addedNodes() {

View File

@ -159,11 +159,7 @@ public class AllocationId implements ToXContentObject, Writeable {
return false;
}
AllocationId that = (AllocationId) o;
if (id.equals(that.id) == false) {
return false;
}
return !(relocationId != null ? !relocationId.equals(that.relocationId) : that.relocationId != null);
return Objects.equals(id, that.id) && Objects.equals(relocationId, that.relocationId);
}
@Override

View File

@ -324,7 +324,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
// be accessible. Therefore, we need to protect against the version being null
// (meaning the node will be going away).
return assignedShards(shardId).stream()
.filter(shr -> !shr.primary() && shr.active())
.filter(shr -> shr.primary() == false && shr.active())
.filter(shr -> node(shr.currentNodeId()) != null)
.max(Comparator.comparing(shr -> node(shr.currentNodeId()).node(),
Comparator.nullsFirst(Comparator.comparing(DiscoveryNode::getVersion))))

View File

@ -71,7 +71,7 @@ public final class ShardRouting implements Writeable, ToXContentObject {
state == ShardRoutingState.RELOCATING : expectedShardSize + " state: " + state;
assert expectedShardSize >= 0 || state != ShardRoutingState.INITIALIZING || state != ShardRoutingState.RELOCATING :
expectedShardSize + " state: " + state;
assert !(state == ShardRoutingState.UNASSIGNED && unassignedInfo == null) : "unassigned shard must be created with meta";
assert (state == ShardRoutingState.UNASSIGNED && unassignedInfo == null) == false : "unassigned shard must be created with meta";
assert (state == ShardRoutingState.UNASSIGNED || state == ShardRoutingState.INITIALIZING) == (recoverySource != null) :
"recovery source only available on unassigned or initializing shard but was " + state;
assert recoverySource == null || recoverySource == PeerRecoverySource.INSTANCE || primary :

View File

@ -244,8 +244,8 @@ public final class UnassignedInfo implements ToXContentFragment, Writeable {
this.failedNodeIds = Collections.unmodifiableSet(failedNodeIds);
assert (failedAllocations > 0) == (reason == Reason.ALLOCATION_FAILED) :
"failedAllocations: " + failedAllocations + " for reason " + reason;
assert !(message == null && failure != null) : "provide a message if a failure exception is provided";
assert !(delayed && reason != Reason.NODE_LEFT) : "shard can only be delayed if it is unassigned due to a node leaving";
assert (message == null && failure != null) == false : "provide a message if a failure exception is provided";
assert (delayed && reason != Reason.NODE_LEFT) == false : "shard can only be delayed if it is unassigned due to a node leaving";
}
public UnassignedInfo(StreamInput in) throws IOException {

View File

@ -742,7 +742,7 @@ public class BalancedShardsAllocator implements ShardsAllocator {
*/
private void allocateUnassigned() {
RoutingNodes.UnassignedShards unassigned = routingNodes.unassigned();
assert !nodes.isEmpty();
assert nodes.isEmpty() == false;
if (logger.isTraceEnabled()) {
logger.trace("Start allocating unassigned shards");
}
@ -1125,7 +1125,7 @@ public class BalancedShardsAllocator implements ShardsAllocator {
public void addShard(ShardRouting shard) {
highestPrimary = -1;
assert !shards.contains(shard) : "Shard already allocated on current node: " + shard;
assert shards.contains(shard) == false : "Shard already allocated on current node: " + shard;
shards.add(shard);
}

View File

@ -92,13 +92,13 @@ public class MoveAllocationCommand implements AllocationCommand {
boolean found = false;
RoutingNode fromRoutingNode = allocation.routingNodes().node(fromDiscoNode.getId());
if (fromRoutingNode == null && !fromDiscoNode.isDataNode()) {
if (fromRoutingNode == null && fromDiscoNode.isDataNode() == false) {
throw new IllegalArgumentException("[move_allocation] can't move [" + index + "][" + shardId + "] from "
+ fromDiscoNode + " to " + toDiscoNode + ": source [" + fromDiscoNode.getName()
+ "] is not a data node.");
}
RoutingNode toRoutingNode = allocation.routingNodes().node(toDiscoNode.getId());
if (toRoutingNode == null && !toDiscoNode.isDataNode()) {
if (toRoutingNode == null && toDiscoNode.isDataNode() == false) {
throw new IllegalArgumentException("[move_allocation] can't move [" + index + "][" + shardId + "] from "
+ fromDiscoNode + " to " + toDiscoNode + ": source [" + toDiscoNode.getName()
+ "] is not a data node.");

View File

@ -32,13 +32,13 @@ public class Classes {
}
public static boolean isInnerClass(Class<?> clazz) {
return !Modifier.isStatic(clazz.getModifiers())
return Modifier.isStatic(clazz.getModifiers()) == false
&& clazz.getEnclosingClass() != null;
}
public static boolean isConcrete(Class<?> clazz) {
int modifiers = clazz.getModifiers();
return !clazz.isInterface() && !Modifier.isAbstract(modifiers);
return clazz.isInterface() == false && Modifier.isAbstract(modifiers) == false;
}
private Classes() {}

View File

@ -32,7 +32,7 @@ public final class FieldMemoryStats implements Writeable, Iterable<ObjectLongCur
*/
public FieldMemoryStats(ObjectLongHashMap<String> stats) {
this.stats = Objects.requireNonNull(stats, "status must be non-null");
assert !stats.containsKey(null);
assert stats.containsKey(null) == false;
}
/**

View File

@ -178,7 +178,7 @@ public class Strings {
* @return <code>true</code> if the CharSequence is either null or has a zero length
*/
public static boolean isEmpty(CharSequence str) {
return !hasLength(str);
return hasLength(str) == false;
}

View File

@ -430,7 +430,7 @@ public final class CopyOnWriteHashMap<K, V> extends AbstractMap<K, V> {
@Override
public boolean hasNext() {
return !entries.isEmpty() || !nodes.isEmpty();
return entries.isEmpty() == false || nodes.isEmpty() == false;
}
@Override
@ -563,7 +563,7 @@ public final class CopyOnWriteHashMap<K, V> extends AbstractMap<K, V> {
@Override
public boolean contains(Object o) {
if (o == null || !(o instanceof Map.Entry)) {
if (o == null || (o instanceof Map.Entry) == false) {
return false;
}
Map.Entry<?, ?> entry = (java.util.Map.Entry<?, ?>) o;

View File

@ -40,7 +40,7 @@ public class Iterators {
@Override
public boolean hasNext() {
boolean hasNext = false;
while (index < iterators.length && !(hasNext = iterators[index].hasNext())) {
while (index < iterators.length && (hasNext = iterators[index].hasNext()) == false) {
index++;
}

View File

@ -217,7 +217,7 @@ public class GeoPolygonDecomposer {
}
// correct the orientation post translation (ccw for shell, cw for holes)
if (component == 0 || (component != 0 && handedness == orientation)) {
orientation = !orientation;
orientation = orientation == false;
}
}
return concat(component, direction ^ orientation, points, offset, edges, toffset, length);
@ -268,8 +268,8 @@ public class GeoPolygonDecomposer {
// ShapeBuilder.intersection that computes dateline edges as valid intersect points
// in support of OGC standards
if (e1.intersect != Edge.MAX_COORDINATE && e2.intersect != Edge.MAX_COORDINATE
&& !(e1.next.next.coordinate.equals(e2.coordinate) && Math.abs(e1.next.coordinate.getX()) == DATELINE
&& Math.abs(e2.coordinate.getX()) == DATELINE)) {
&& (e1.next.next.coordinate.equals(e2.coordinate) && Math.abs(e1.next.coordinate.getX()) == DATELINE
&& Math.abs(e2.coordinate.getX()) == DATELINE) == false) {
connect(e1, e2);
}
}
@ -365,12 +365,12 @@ public class GeoPolygonDecomposer {
*/
private static int intersections(double dateline, Edge[] edges) {
int numIntersections = 0;
assert !Double.isNaN(dateline);
assert Double.isNaN(dateline) == false;
int maxComponent = 0;
for (int i = 0; i < edges.length; i++) {
Point p1 = edges[i].coordinate;
Point p2 = edges[i].next.coordinate;
assert !Double.isNaN(p2.getX()) && !Double.isNaN(p1.getX());
assert Double.isNaN(p2.getX()) == false && Double.isNaN(p1.getX()) == false;
edges[i].intersect = Edge.MAX_COORDINATE;
double position = intersection(p1.getX(), p2.getX(), dateline);
@ -475,7 +475,7 @@ public class GeoPolygonDecomposer {
final int pos;
boolean sharedVertex = false;
if (((pos = Arrays.binarySearch(edges, 0, intersections, current, INTERSECTION_ORDER)) >= 0)
&& !(sharedVertex = (edges[pos].intersect.equals(current.coordinate)))) {
&& (sharedVertex = (edges[pos].intersect.equals(current.coordinate))) == false) {
// The binary search returned an exact match, but we checked again using compareTo()
// and it didn't match after all.

View File

@ -474,7 +474,7 @@ public class GeoUtils {
}
}
if (geohash != null) {
if(!Double.isNaN(lat) || !Double.isNaN(lon)) {
if(Double.isNaN(lat) == false || Double.isNaN(lon) == false) {
throw new ElasticsearchParseException("field must be either lat/lon or geohash");
} else {
return point.parseGeoHash(geohash, effectivePoint);

View File

@ -139,7 +139,7 @@ public class LineStringBuilder extends ShapeBuilder<JtsGeometry, org.elasticsear
for (int i = 1; i < coordinates.length; i++) {
double t = intersection(coordinates[i-1], coordinates[i], dateline);
if(!Double.isNaN(t)) {
if(Double.isNaN(t) == false) {
Coordinate[] part;
if(t<1) {
part = Arrays.copyOfRange(coordinates, offset, i+1);

View File

@ -557,7 +557,7 @@ public class PolygonBuilder extends ShapeBuilder<JtsGeometry, org.elasticsearch.
final int pos;
boolean sharedVertex = false;
if (((pos = Arrays.binarySearch(edges, 0, intersections, current, INTERSECTION_ORDER)) >= 0)
&& !(sharedVertex = (edges[pos].intersect.compareTo(current.coordinate) == 0))) {
&& (sharedVertex = (edges[pos].intersect.compareTo(current.coordinate) == 0)) == false) {
// The binary search returned an exact match, but we checked again using compareTo()
// and it didn't match after all.
@ -626,8 +626,8 @@ public class PolygonBuilder extends ShapeBuilder<JtsGeometry, org.elasticsearch.
// ShapeBuilder.intersection that computes dateline edges as valid intersect points
// in support of OGC standards
if (e1.intersect != Edge.MAX_COORDINATE && e2.intersect != Edge.MAX_COORDINATE
&& !(e1.next.next.coordinate.equals3D(e2.coordinate) && Math.abs(e1.next.coordinate.x) == DATELINE
&& Math.abs(e2.coordinate.x) == DATELINE) ) {
&& (e1.next.next.coordinate.equals3D(e2.coordinate) && Math.abs(e1.next.coordinate.x) == DATELINE
&& Math.abs(e2.coordinate.x) == DATELINE) == false ) {
connect(e1, e2);
}
}
@ -729,7 +729,7 @@ public class PolygonBuilder extends ShapeBuilder<JtsGeometry, org.elasticsearch.
}
// correct the orientation post translation (ccw for shell, cw for holes)
if (component == 0 || (component != 0 && handedness == orientation)) {
orientation = !orientation;
orientation = orientation == false;
}
}
return concat(component, direction ^ orientation, points, offset, edges, toffset, length);
@ -763,7 +763,7 @@ public class PolygonBuilder extends ShapeBuilder<JtsGeometry, org.elasticsearch.
if (direction) {
edges[edgeOffset + i] = new Edge(points[pointOffset + i], edges[edgeOffset + i - 1]);
edges[edgeOffset + i].component = component;
} else if(!edges[edgeOffset + i - 1].coordinate.equals(points[pointOffset + i])) {
} else if (edges[edgeOffset + i - 1].coordinate.equals(points[pointOffset + i]) == false) {
edges[edgeOffset + i - 1].next = edges[edgeOffset + i] = new Edge(points[pointOffset + i], null);
edges[edgeOffset + i - 1].component = component;
} else {

View File

@ -261,12 +261,12 @@ public abstract class ShapeBuilder<T extends Shape, G extends org.elasticsearch.
*/
protected static int intersections(double dateline, Edge[] edges) {
int numIntersections = 0;
assert !Double.isNaN(dateline);
assert Double.isNaN(dateline) == false;
int maxComponent = 0;
for (int i = 0; i < edges.length; i++) {
Coordinate p1 = edges[i].coordinate;
Coordinate p2 = edges[i].next.coordinate;
assert !Double.isNaN(p2.x) && !Double.isNaN(p1.x);
assert Double.isNaN(p2.x) == false && Double.isNaN(p1.x) == false;
edges[i].intersect = Edge.MAX_COORDINATE;
double position = intersection(p1, p2, dateline);

View File

@ -256,7 +256,7 @@ class BindingProcessor extends AbstractProcessor {
}
Binding<?> original = injector.state.getExplicitBinding(key);
if (original != null && !isOkayDuplicate(original, binding)) {
if (original != null && isOkayDuplicate(original, binding) == false) {
errors.bindingAlreadySet(key, original.getSource());
return;
}

View File

@ -147,7 +147,7 @@ class InjectorImpl implements Injector, Lookups {
*/
static boolean isMembersInjector(Key<?> key) {
return key.getTypeLiteral().getRawType().equals(MembersInjector.class)
&& !key.hasAnnotationType();
&& key.hasAnnotationType() == false;
}
private <T> BindingImpl<MembersInjector<T>> createMembersInjectorBinding(
@ -248,7 +248,7 @@ class InjectorImpl implements Injector, Lookups {
// Find a constant string binding.
Key<String> stringKey = key.ofType(String.class);
BindingImpl<String> stringBinding = state.getExplicitBinding(stringKey);
if (stringBinding == null || !stringBinding.isConstant()) {
if (stringBinding == null || stringBinding.isConstant() == false) {
return null;
}
@ -480,7 +480,7 @@ class InjectorImpl implements Injector, Lookups {
errors, context, dependency);
try {
Object o = provider.get();
if (o != null && !rawType.isInstance(o)) {
if (o != null && rawType.isInstance(o) == false) {
throw errors.subtypeNotProvided(providerType, rawType).toException();
}
@SuppressWarnings("unchecked") // protected by isInstance() check above

View File

@ -57,7 +57,7 @@ class MembersInjectorStore {
* aren't any type listeners.
*/
public boolean hasTypeListeners() {
return !typeListenerBindings.isEmpty();
return typeListenerBindings.isEmpty() == false;
}
/**

View File

@ -106,7 +106,7 @@ public abstract class AbstractBindingBuilder<T> {
}
protected boolean keyTypeIsSet() {
return !Void.class.equals(binding.getKey().getTypeLiteral().getType());
return Void.class.equals(binding.getKey().getTypeLiteral().getType()) == false;
}
protected void checkNotTargetted() {

View File

@ -476,7 +476,7 @@ public class MoreTypes {
@Override
public boolean isFullySpecified() {
if (ownerType != null && !MoreTypes.isFullySpecified(ownerType)) {
if (ownerType != null && MoreTypes.isFullySpecified(ownerType) == false) {
return false;
}

View File

@ -75,7 +75,7 @@ public class Matchers {
@Override
public boolean matches(T t) {
return !delegate.matches(t);
return delegate.matches(t) == false;
}
@Override

View File

@ -258,7 +258,7 @@ public abstract class MapBinder<K, V> {
@Override
public LinkedBindingBuilder<V> addBinding(K key) {
Multibinder.checkNotNull(key, "key");
Multibinder.checkConfiguration(!isInitialized(), "MapBinder was already initialized");
Multibinder.checkConfiguration(isInitialized() == false, "MapBinder was already initialized");
Key<V> valueKey = Key.get(valueType, new RealElement(entrySetBinder.getSetName()));
entrySetBinder.addBinding().toInstance(new MapEntry<>(key,
@ -309,7 +309,7 @@ public abstract class MapBinder<K, V> {
@Override @SuppressWarnings({"rawtypes", "unchecked"}) // code is silly stupid with generics
public void configure(Binder binder) {
Multibinder.checkConfiguration(!isInitialized(), "MapBinder was already initialized");
Multibinder.checkConfiguration(isInitialized() == false, "MapBinder was already initialized");
final Set<Dependency<?>> dependencies = singleton(Dependency.get(entrySetBinder.getSetKey()));

View File

@ -221,14 +221,14 @@ public abstract class Multibinder<T> {
@Override
public void configure(Binder binder) {
checkConfiguration(!isInitialized(), "Multibinder was already initialized");
checkConfiguration(isInitialized() == false, "Multibinder was already initialized");
binder.bind(setKey).toProvider(this);
}
@Override
public LinkedBindingBuilder<T> addBinding() {
checkConfiguration(!isInitialized(), "Multibinder was already initialized");
checkConfiguration(isInitialized() == false, "Multibinder was already initialized");
return binder.bind(Key.get(elementType, new RealElement(setName)));
}

View File

@ -216,7 +216,7 @@ public final class InjectionPoint {
// Disallow private constructors on non-private classes (unless they have @Inject)
if (Modifier.isPrivate(noArgConstructor.getModifiers())
&& !Modifier.isPrivate(rawType.getModifiers())) {
&& Modifier.isPrivate(rawType.getModifiers()) == false) {
errors.missingConstructor(rawType);
throw new ConfigurationException(errors.getMessages());
}

View File

@ -1112,7 +1112,7 @@ public final class Settings implements ToXContentFragment {
@Override
public boolean shouldRemoveMissingPlaceholder(String placeholderName) {
return !placeholderName.startsWith("prompt.");
return placeholderName.startsWith("prompt.") == false;
}
};

View File

@ -40,7 +40,7 @@ public class PortsRange {
public boolean iterate(PortCallback callback) throws NumberFormatException {
StringTokenizer st = new StringTokenizer(portRange, ",");
boolean success = false;
while (st.hasMoreTokens() && !success) {
while (st.hasMoreTokens() && success == false) {
String portToken = st.nextToken().trim();
int index = portToken.indexOf('-');
if (index == -1) {

View File

@ -85,7 +85,7 @@ public class CancellableThreads {
assert cancelled : "Interruption via Thread#interrupt() is unsupported. Use CancellableThreads#cancel() instead";
// we can only reach here if assertions are disabled. If we reach this code and cancelled is false, this means that we've
// been interrupted externally (which we don't support).
cancelledByExternalInterrupt = !cancelled;
cancelledByExternalInterrupt = cancelled == false;
} catch (RuntimeException t) {
runtimeException = t;
} finally {

View File

@ -207,7 +207,7 @@ public class PrioritizedEsThreadPoolExecutor extends EsThreadPoolExecutor {
@Override
public int compareTo(PrioritizedRunnable pr) {
int res = super.compareTo(pr);
if (res != 0 || !(pr instanceof TieBreakingPrioritizedRunnable)) {
if (res != 0 || (pr instanceof TieBreakingPrioritizedRunnable) == false) {
return res;
}
return insertionOrder < ((TieBreakingPrioritizedRunnable) pr).insertionOrder ? -1 : 1;

View File

@ -74,7 +74,7 @@ public final class Sets {
public static <T> Set<T> difference(Set<T> left, Set<T> right) {
Objects.requireNonNull(left);
Objects.requireNonNull(right);
return left.stream().filter(k -> !right.contains(k)).collect(Collectors.toSet());
return left.stream().filter(k -> right.contains(k) == false).collect(Collectors.toSet());
}
/**

View File

@ -198,7 +198,7 @@ public class XContentHelper {
public static String convertToJson(BytesReference bytes, boolean reformatJson, boolean prettyPrint, XContentType xContentType)
throws IOException {
Objects.requireNonNull(xContentType);
if (xContentType.canonical() == XContentType.JSON && !reformatJson) {
if (xContentType.canonical() == XContentType.JSON && reformatJson == false) {
return bytes.utf8ToString();
}
@ -251,7 +251,7 @@ public class XContentHelper {
if (old instanceof Map && changesEntry.getValue() instanceof Map) {
// recursive merge maps
modified |= update((Map<String, Object>) source.get(changesEntry.getKey()),
(Map<String, Object>) changesEntry.getValue(), checkUpdatesAreUnequal && !modified);
(Map<String, Object>) changesEntry.getValue(), checkUpdatesAreUnequal && modified == false);
continue;
}
// update the field
@ -263,7 +263,7 @@ public class XContentHelper {
modified = true;
continue;
}
modified = !Objects.equals(old, changesEntry.getValue());
modified = Objects.equals(old, changesEntry.getValue()) == false;
}
return modified;
}

View File

@ -68,7 +68,7 @@ public class BlockingClusterStatePublishResponseHandler {
*/
public boolean awaitAllNodes(TimeValue timeout) throws InterruptedException {
boolean success = latch.await(timeout.millis(), TimeUnit.MILLISECONDS);
assert !success || pendingNodes.isEmpty() : "response count reached 0 but still waiting for some nodes";
assert success == false || pendingNodes.isEmpty() : "response count reached 0 but still waiting for some nodes";
return success;
}

View File

@ -243,7 +243,7 @@ public abstract class AsyncShardFetch<T extends BaseNodeResponse> implements Rel
}
}
// remove nodes that are not longer part of the data nodes set
shardCache.keySet().removeIf(nodeId -> !nodes.nodeExists(nodeId));
shardCache.keySet().removeIf(nodeId -> nodes.nodeExists(nodeId) == false);
}
/**

View File

@ -77,7 +77,7 @@ public class Gateway {
// update the global state, and clean the indices, we elect them in the next phase
final Metadata.Builder metadataBuilder = Metadata.builder(electedGlobalState).removeAllIndices();
assert !indices.containsKey(null);
assert indices.containsKey(null) == false;
final Object[] keys = indices.keys;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {

View File

@ -155,7 +155,7 @@ public abstract class IndexNumericFieldData implements IndexFieldData<LeafNumeri
case DATE_NANOSECONDS:
return dateNanosComparatorSource(missingValue, sortMode, nested);
default:
assert !targetNumericType.isFloatingPoint();
assert targetNumericType.isFloatingPoint() == false;
return new LongValuesComparatorSource(this, missingValue, sortMode, nested);
}
}

View File

@ -474,7 +474,7 @@ public class CompletionFieldMapper extends FieldMapper {
contextMapping = contextMappings.get(fieldName);
} else {
assert fieldName != null;
assert !contextsMap.containsKey(fieldName);
assert contextsMap.containsKey(fieldName) == false;
contextsMap.put(fieldName, contextMapping.parseContext(parseContext, parser));
}
}

View File

@ -417,7 +417,7 @@ public final class DateFieldMapper extends FieldMapper {
if (lowerTerm == null) {
l = Long.MIN_VALUE;
} else {
l = parseToLong(lowerTerm, !includeLower, timeZone, parser, nowSupplier, resolution);
l = parseToLong(lowerTerm, includeLower == false, timeZone, parser, nowSupplier, resolution);
if (includeLower == false) {
++l;
}
@ -503,7 +503,7 @@ public final class DateFieldMapper extends FieldMapper {
long fromInclusive = Long.MIN_VALUE;
if (from != null) {
fromInclusive = parseToLong(from, !includeLower, timeZone, dateParser, context::nowInMillis, resolution);
fromInclusive = parseToLong(from, includeLower == false, timeZone, dateParser, context::nowInMillis, resolution);
if (includeLower == false) {
if (fromInclusive == Long.MAX_VALUE) {
return Relation.DISJOINT;

View File

@ -248,10 +248,10 @@ public class DynamicTemplate implements ToXContentObject {
}
public boolean match(String path, String name, XContentFieldType xcontentFieldType) {
if (pathMatch != null && !matchType.matches(pathMatch, path)) {
if (pathMatch != null && matchType.matches(pathMatch, path) == false) {
return false;
}
if (match != null && !matchType.matches(match, name)) {
if (match != null && matchType.matches(match, name) == false) {
return false;
}
if (pathUnmatch != null && matchType.matches(pathUnmatch, path)) {

View File

@ -92,7 +92,7 @@ public class RootObjectMapper extends ObjectMapper {
@Override
protected ObjectMapper createMapper(String name, String fullPath, Explicit<Boolean> enabled, Nested nested, Dynamic dynamic,
Map<String, Mapper> mappers, Version indexCreatedVersion) {
assert !nested.isNested();
assert nested.isNested() == false;
return new RootObjectMapper(name, enabled, dynamic, mappers, runtimeFieldTypes,
dynamicDateTimeFormatters,
dynamicTemplates,

View File

@ -224,7 +224,7 @@ public class BoolQueryBuilder extends AbstractQueryBuilder<BoolQueryBuilder> {
* Otherwise <code>false</code>.
*/
public boolean hasClauses() {
return !(mustClauses.isEmpty() && shouldClauses.isEmpty() && mustNotClauses.isEmpty() && filterClauses.isEmpty());
return (mustClauses.isEmpty() && shouldClauses.isEmpty() && mustNotClauses.isEmpty() && filterClauses.isEmpty()) == false;
}
/**

View File

@ -319,8 +319,8 @@ public class GeoBoundingBoxQueryBuilder extends AbstractQueryBuilder<GeoBounding
double left = luceneTopLeft.getLon();
boolean completeLonRange = ((right - left) % 360 == 0 && right > left);
GeoUtils.normalizePoint(luceneTopLeft, true, !completeLonRange);
GeoUtils.normalizePoint(luceneBottomRight, true, !completeLonRange);
GeoUtils.normalizePoint(luceneTopLeft, true, completeLonRange == false);
GeoUtils.normalizePoint(luceneBottomRight, true, completeLonRange == false);
if (completeLonRange) {
luceneTopLeft.resetLon(-180);
luceneBottomRight.resetLon(180);

View File

@ -207,7 +207,7 @@ public class SpanNearQueryBuilder extends AbstractQueryBuilder<SpanNearQueryBuil
assert query instanceof SpanQuery;
}
if (clauses.size() == 1) {
assert !isGap;
assert isGap == false;
return query;
}
String spanNearFieldName = null;

View File

@ -76,7 +76,14 @@ public final class QueryParserHelper {
boolean allField = Regex.isMatchAllPattern(fieldEntry.getKey());
boolean multiField = Regex.isSimpleMatchPattern(fieldEntry.getKey());
float weight = fieldEntry.getValue() == null ? 1.0f : fieldEntry.getValue();
Map<String, Float> fieldMap = resolveMappingField(context, fieldEntry.getKey(), weight, !multiField, !allField, fieldSuffix);
Map<String, Float> fieldMap = resolveMappingField(
context,
fieldEntry.getKey(),
weight,
multiField == false,
allField == false,
fieldSuffix
);
for (Map.Entry<String, Float> field : fieldMap.entrySet()) {
float boost = field.getValue();

View File

@ -270,7 +270,14 @@ public class QueryStringQueryParser extends XQueryParser {
boolean multiFields = Regex.isSimpleMatchPattern(field);
// Filters unsupported fields if a pattern is requested
// Filters metadata fields if all fields are requested
extractedFields = resolveMappingField(context, field, 1.0f, !allFields, !multiFields, quoted ? quoteFieldSuffix : null);
extractedFields = resolveMappingField(
context,
field,
1.0f,
allFields == false,
multiFields == false,
quoted ? quoteFieldSuffix : null
);
} else if (quoted && quoteFieldSuffix != null) {
extractedFields = resolveMappingFields(context, fieldsAndWeights, quoteFieldSuffix);
} else {

View File

@ -261,7 +261,7 @@ public class SearchStats implements Writeable, ToXContentFragment {
}
addTotals(searchStats);
openContexts += searchStats.openContexts;
if (searchStats.groupStats != null && !searchStats.groupStats.isEmpty()) {
if (searchStats.groupStats != null && searchStats.groupStats.isEmpty() == false) {
if (groupStats == null) {
groupStats = new HashMap<>(searchStats.groupStats.size());
}
@ -304,7 +304,7 @@ public class SearchStats implements Writeable, ToXContentFragment {
builder.startObject(Fields.SEARCH);
builder.field(Fields.OPEN_CONTEXTS, openContexts);
totalStats.toXContent(builder, params);
if (groupStats != null && !groupStats.isEmpty()) {
if (groupStats != null && groupStats.isEmpty() == false) {
builder.startObject(Fields.GROUPS);
for (Map.Entry<String, Stats> entry : groupStats.entrySet()) {
builder.startObject(entry.getKey());

View File

@ -757,38 +757,38 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
assert primaryMode || checkpoints.values().stream().allMatch(cps -> cps.globalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO);
// relocation handoff can only occur in primary mode
assert !handoffInProgress || primaryMode;
assert handoffInProgress == false || primaryMode;
// a relocated copy is not in primary mode
assert !relocated || !primaryMode;
assert relocated == false || primaryMode == false;
// the current shard is marked as in-sync when the global checkpoint tracker operates in primary mode
assert !primaryMode || checkpoints.get(shardAllocationId).inSync;
assert primaryMode == false || checkpoints.get(shardAllocationId).inSync;
// the routing table and replication group is set when the global checkpoint tracker operates in primary mode
assert !primaryMode || (routingTable != null && replicationGroup != null) :
assert primaryMode == false || (routingTable != null && replicationGroup != null) :
"primary mode but routing table is " + routingTable + " and replication group is " + replicationGroup;
// when in primary mode, the current allocation ID is the allocation ID of the primary or the relocation allocation ID
assert !primaryMode
assert primaryMode == false
|| (routingTable.primaryShard().allocationId().getId().equals(shardAllocationId)
|| routingTable.primaryShard().allocationId().getRelocationId().equals(shardAllocationId));
// during relocation handoff there are no entries blocking global checkpoint advancement
assert !handoffInProgress || pendingInSync.isEmpty() :
assert handoffInProgress == false || pendingInSync.isEmpty() :
"entries blocking global checkpoint advancement during relocation handoff: " + pendingInSync;
// entries blocking global checkpoint advancement can only exist in primary mode and when not having a relocation handoff
assert pendingInSync.isEmpty() || (primaryMode && !handoffInProgress);
// the computed global checkpoint is always up-to-date
assert !primaryMode
assert primaryMode == false
|| globalCheckpoint == computeGlobalCheckpoint(pendingInSync, checkpoints.values(), globalCheckpoint)
: "global checkpoint is not up-to-date, expected: " +
computeGlobalCheckpoint(pendingInSync, checkpoints.values(), globalCheckpoint) + " but was: " + globalCheckpoint;
// when in primary mode, the global checkpoint is at most the minimum local checkpoint on all in-sync shard copies
assert !primaryMode
assert primaryMode == false
|| globalCheckpoint <= inSyncCheckpointStates(checkpoints, CheckpointState::getLocalCheckpoint, LongStream::min)
: "global checkpoint [" + globalCheckpoint + "] "
+ "for primary mode allocation ID [" + shardAllocationId + "] "
@ -807,10 +807,10 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
for (Map.Entry<String, CheckpointState> entry : checkpoints.entrySet()) {
// blocking global checkpoint advancement only happens for shards that are not in-sync
assert !pendingInSync.contains(entry.getKey()) || !entry.getValue().inSync :
assert pendingInSync.contains(entry.getKey()) == false || entry.getValue().inSync == false :
"shard copy " + entry.getKey() + " blocks global checkpoint advancement but is in-sync";
// in-sync shard copies are tracked
assert !entry.getValue().inSync || entry.getValue().tracked :
assert entry.getValue().inSync == false || entry.getValue().tracked :
"shard copy " + entry.getKey() + " is in-sync but not tracked";
}
@ -994,7 +994,7 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
assert handoffInProgress == false;
assert invariant();
final CheckpointState cps = checkpoints.get(allocationId);
assert !this.shardAllocationId.equals(allocationId) || cps != null;
assert this.shardAllocationId.equals(allocationId) == false || cps != null;
if (cps != null && globalCheckpoint > cps.globalCheckpoint) {
final long previousGlobalCheckpoint = cps.globalCheckpoint;
cps.globalCheckpoint = globalCheckpoint;
@ -1073,7 +1073,7 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
assert invariant();
if (applyingClusterStateVersion > appliedClusterStateVersion) {
// check that the master does not fabricate new in-sync entries out of thin air once we are in primary mode
assert !primaryMode || inSyncAllocationIds.stream().allMatch(
assert primaryMode == false || inSyncAllocationIds.stream().allMatch(
inSyncId -> checkpoints.containsKey(inSyncId) && checkpoints.get(inSyncId).inSync) :
"update from master in primary mode contains in-sync ids " + inSyncAllocationIds +
" that have no matching entries in " + checkpoints;
@ -1081,7 +1081,7 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
Set<String> initializingAllocationIds = routingTable.getAllInitializingShards().stream()
.map(ShardRouting::allocationId).map(AllocationId::getId).collect(Collectors.toSet());
boolean removedEntries = checkpoints.keySet().removeIf(
aid -> !inSyncAllocationIds.contains(aid) && !initializingAllocationIds.contains(aid));
aid -> inSyncAllocationIds.contains(aid) == false && initializingAllocationIds.contains(aid) == false);
if (primaryMode) {
// add new initializingIds that are missing locally. These are fresh shard copies - and not in-sync
@ -1165,7 +1165,7 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
updateLocalCheckpoint(allocationId, cps, localCheckpoint);
// if it was already in-sync (because of a previously failed recovery attempt), global checkpoint must have been
// stuck from advancing
assert !cps.inSync || (cps.localCheckpoint >= getGlobalCheckpoint()) :
assert cps.inSync == false || (cps.localCheckpoint >= getGlobalCheckpoint()) :
"shard copy " + allocationId + " that's already in-sync should have a local checkpoint " + cps.localCheckpoint +
" that's above the global checkpoint " + getGlobalCheckpoint();
if (cps.localCheckpoint < getGlobalCheckpoint()) {

View File

@ -1229,11 +1229,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
source.equals(index.source) == false) {
return false;
}
if (routing != null ? !routing.equals(index.routing) : index.routing != null) {
return false;
}
return true;
return Objects.equals(routing, index.routing);
}
@Override

View File

@ -123,7 +123,7 @@ public class IndicesStore implements ClusterStateListener, Closeable {
// remove entries from cache that don't exist in the routing table anymore (either closed or deleted indices)
// - removing shard data of deleted indices is handled by IndicesClusterStateService
// - closed indices don't need to be removed from the cache but we do it anyway for code simplicity
folderNotFoundCache.removeIf(shardId -> !routingTable.hasIndex(shardId.getIndex()));
folderNotFoundCache.removeIf(shardId -> routingTable.hasIndex(shardId.getIndex()) == false);
// remove entries from cache which are allocated to this node
final String localNodeId = event.state().nodes().getLocalNodeId();
RoutingNode localRoutingNode = event.state().getRoutingNodes().node(localNodeId);

View File

@ -66,7 +66,7 @@ public class DeadlockAnalyzer {
Set<LinkedHashSet<ThreadInfo>> cycles = new HashSet<>();
for (Map.Entry<Long, ThreadInfo> entry : threadInfoMap.entrySet()) {
LinkedHashSet<ThreadInfo> cycle = new LinkedHashSet<>();
for (ThreadInfo t = entry.getValue(); !cycle.contains(t); t = threadInfoMap.get(Long.valueOf(t.getLockOwnerId()))) {
for (ThreadInfo t = entry.getValue(); cycle.contains(t) == false; t = threadInfoMap.get(Long.valueOf(t.getLockOwnerId()))) {
cycle.add(t);
}

View File

@ -289,7 +289,7 @@ public class OsProbe {
@SuppressForbidden(reason = "access /proc/self/cgroup")
List<String> readProcSelfCgroup() throws IOException {
final List<String> lines = Files.readAllLines(PathUtils.get("/proc/self/cgroup"));
assert lines != null && !lines.isEmpty();
assert lines != null && lines.isEmpty() == false;
return lines;
}
@ -517,7 +517,7 @@ public class OsProbe {
return null;
} else {
final Map<String, String> controllerMap = getControlGroups();
assert !controllerMap.isEmpty();
assert controllerMap.isEmpty() == false;
final String cpuAcctControlGroup = controllerMap.get("cpuacct");
if (cpuAcctControlGroup == null) {

View File

@ -70,8 +70,10 @@ public abstract class BaseRestHandler implements RestHandler {
// validate unconsumed params, but we must exclude params used to format the response
// use a sorted set so the unconsumed parameters appear in a reliable sorted order
final SortedSet<String> unconsumedParams =
request.unconsumedParams().stream().filter(p -> !responseParams().contains(p)).collect(Collectors.toCollection(TreeSet::new));
final SortedSet<String> unconsumedParams = request.unconsumedParams()
.stream()
.filter(p -> responseParams().contains(p) == false)
.collect(Collectors.toCollection(TreeSet::new));
// validate the non-response params
if (unconsumedParams.isEmpty() == false) {

View File

@ -344,7 +344,7 @@ public class RestRequest implements ToXContent.Params {
return params
.keySet()
.stream()
.filter(p -> !consumedParams.contains(p))
.filter(p -> consumedParams.contains(p) == false)
.collect(Collectors.toList());
}

View File

@ -64,7 +64,7 @@ public class RestNodesInfoAction extends BaseRestHandler {
// special case like /_nodes/os (in this case os are metrics and not the nodeId)
// still, /_nodes/_local (or any other node id) should work and be treated as usual
// this means one must differentiate between allowed metrics and arbitrary node ids in the same place
if (request.hasParam("nodeId") && !request.hasParam("metrics")) {
if (request.hasParam("nodeId") && request.hasParam("metrics") == false) {
String nodeId = request.param("nodeId", "_all");
Set<String> metricsOrNodeIds = Strings.tokenizeByCommaToSet(nodeId);
boolean isMetricsOnly = ALLOWED_METRICS.containsAll(metricsOrNodeIds);

View File

@ -60,7 +60,7 @@ public class RestGetSourceAction extends BaseRestHandler {
getRequest.fetchSourceContext(FetchSourceContext.parseFromRestRequest(request));
return channel -> {
if (getRequest.fetchSourceContext() != null && !getRequest.fetchSourceContext().fetchSource()) {
if (getRequest.fetchSourceContext() != null && getRequest.fetchSourceContext().fetchSource() == false) {
final ActionRequestValidationException validationError = new ActionRequestValidationException();
validationError.addValidationError("fetching source can not be disabled");
channel.sendResponse(new BytesRestResponse(channel, validationError));

View File

@ -56,7 +56,7 @@ class TermsSortedDocsProducer extends SortedDocsProducer {
break;
}
reuse = te.postings(reuse, PostingsEnum.NONE);
if (processBucket(queue, context, reuse, te.term(), builder) && !first) {
if (processBucket(queue, context, reuse, te.term(), builder) && first == false) {
// this bucket does not have any competitive composite buckets,
// we can early terminate the collection because the remaining buckets are guaranteed
// to be greater than this bucket.

View File

@ -159,7 +159,7 @@ public class AutoDateHistogramAggregationBuilder extends ValuesSourceAggregation
}
public AutoDateHistogramAggregationBuilder setMinimumIntervalExpression(String minimumIntervalExpression) {
if (minimumIntervalExpression != null && !ALLOWED_INTERVALS.containsValue(minimumIntervalExpression)) {
if (minimumIntervalExpression != null && ALLOWED_INTERVALS.containsValue(minimumIntervalExpression) == false) {
throw new IllegalArgumentException(MINIMUM_INTERVAL_FIELD.getPreferredName() +
" must be one of [" + ALLOWED_INTERVALS.values().toString() + "]");
}

View File

@ -229,6 +229,6 @@ public abstract class TermsAggregator extends DeferableBucketAggregator {
@Override
protected boolean shouldDefer(Aggregator aggregator) {
return collectMode == SubAggCollectionMode.BREADTH_FIRST
&& !aggsUsedForSorting.contains(aggregator);
&& aggsUsedForSorting.contains(aggregator) == false;
}
}

View File

@ -68,7 +68,7 @@ public class InternalMax extends InternalNumericMetricsAggregation.SingleValue i
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
boolean hasValue = !Double.isInfinite(max);
boolean hasValue = Double.isInfinite(max) == false;
builder.field(CommonFields.VALUE.getPreferredName(), hasValue ? max : null);
if (hasValue && format != DocValueFormat.RAW) {
builder.field(CommonFields.VALUE_AS_STRING.getPreferredName(), format.format(max).toString());

View File

@ -68,7 +68,7 @@ public class InternalMin extends InternalNumericMetricsAggregation.SingleValue i
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
boolean hasValue = !Double.isInfinite(min);
boolean hasValue = Double.isInfinite(min) == false;
builder.field(CommonFields.VALUE.getPreferredName(), hasValue ? min : null);
if (hasValue && format != DocValueFormat.RAW) {
builder.field(CommonFields.VALUE_AS_STRING.getPreferredName(), format.format(min).toString());

View File

@ -28,7 +28,7 @@ public class ParsedMax extends ParsedSingleValueNumericMetricsAggregation implem
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
boolean hasValue = !Double.isInfinite(value);
boolean hasValue = Double.isInfinite(value) == false;
builder.field(CommonFields.VALUE.getPreferredName(), hasValue ? value : null);
if (hasValue && valueAsString != null) {
builder.field(CommonFields.VALUE_AS_STRING.getPreferredName(), valueAsString);

View File

@ -28,7 +28,7 @@ public class ParsedMin extends ParsedSingleValueNumericMetricsAggregation implem
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
boolean hasValue = !Double.isInfinite(value);
boolean hasValue = Double.isInfinite(value) == false;
builder.field(CommonFields.VALUE.getPreferredName(), hasValue ? value : null);
if (hasValue && valueAsString != null) {
builder.field(CommonFields.VALUE_AS_STRING.getPreferredName(), valueAsString);

View File

@ -47,7 +47,7 @@ public abstract class BucketMetricsPipelineAggregator extends SiblingPipelineAgg
List<? extends InternalMultiBucketAggregation.InternalBucket> buckets = multiBucketsAgg.getBuckets();
for (InternalMultiBucketAggregation.InternalBucket bucket : buckets) {
Double bucketValue = BucketHelpers.resolveBucketValue(multiBucketsAgg, bucket, sublistedPath, gapPolicy);
if (bucketValue != null && !Double.isNaN(bucketValue)) {
if (bucketValue != null && Double.isNaN(bucketValue) == false) {
collectBucketValue(bucket.getKeyAsString(), bucketValue);
}
}

View File

@ -92,7 +92,7 @@ public class InternalBucketMetricValue extends InternalNumericMetricsAggregation
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
boolean hasValue = !Double.isInfinite(value);
boolean hasValue = Double.isInfinite(value) == false;
builder.field(CommonFields.VALUE.getPreferredName(), hasValue ? value : null);
if (hasValue && format != DocValueFormat.RAW) {
builder.field(CommonFields.VALUE_AS_STRING.getPreferredName(), format.format(value).toString());

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