Replace NOT operator with explicit false check (#68124)

Part 3.

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-01-29 13:51:41 +00:00 committed by Rory Hunter
parent 78c6d73a65
commit 6c8ed22e95
101 changed files with 196 additions and 192 deletions

View file

@ -97,7 +97,7 @@ public class HotThreadsIT extends ESIntegTestCase {
}
success = true;
} finally {
if (!success) {
if (success == false) {
hasErrors.set(true);
}
latch.countDown();

View file

@ -529,7 +529,7 @@ public class BulkWithUpdatesIT extends ESIntegTestCase {
int successes = 0;
for (BulkResponse response : responses) {
if (!response.hasFailures()) {
if (response.hasFailures() == false) {
successes++;
}
}

View file

@ -219,11 +219,11 @@ public class GetTermVectorsIT extends AbstractTermVectorsTestCase {
}
public static String termVectorOptionsToString(FieldType fieldType) {
if (!fieldType.storeTermVectors()) {
if (fieldType.storeTermVectors() == false) {
return "no";
} else if (!fieldType.storeTermVectorOffsets() && !fieldType.storeTermVectorPositions()) {
} else if (fieldType.storeTermVectorOffsets() == false && fieldType.storeTermVectorPositions() == false) {
return "yes";
} else if (fieldType.storeTermVectorOffsets() && !fieldType.storeTermVectorPositions()) {
} else if (fieldType.storeTermVectorOffsets() && fieldType.storeTermVectorPositions() == false) {
return "with_offsets";
} else {
StringBuilder builder = new StringBuilder("with");

View file

@ -49,7 +49,7 @@ public class SimpleAllocationIT extends ESIntegTestCase {
ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(0));
for (RoutingNode node : state.getRoutingNodes()) {
if (!node.isEmpty()) {
if (node.isEmpty() == false) {
assertThat(node.size(), equalTo(2));
}
}
@ -60,7 +60,7 @@ public class SimpleAllocationIT extends ESIntegTestCase {
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(0));
for (RoutingNode node : state.getRoutingNodes()) {
if (!node.isEmpty()) {
if (node.isEmpty() == false) {
assertThat(node.size(), equalTo(1));
}
}
@ -79,7 +79,7 @@ public class SimpleAllocationIT extends ESIntegTestCase {
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(0));
for (RoutingNode node : state.getRoutingNodes()) {
if (!node.isEmpty()) {
if (node.isEmpty() == false) {
assertThat(node.size(), equalTo(4));
}
}

View file

@ -46,8 +46,7 @@ public class RecoverAfterNodesIT extends ESIntegTestCase {
do {
blocks = nodeClient.admin().cluster().prepareState().setLocal(true).execute().actionGet()
.getState().blocks().global(ClusterBlockLevel.METADATA_WRITE);
}
while (!blocks.isEmpty() && (System.currentTimeMillis() - start) < timeout.millis());
} while (blocks.isEmpty() == false && (System.currentTimeMillis() - start) < timeout.millis());
return blocks;
}

View file

@ -210,14 +210,14 @@ public class CorruptedFileIT extends ESIntegTestCase {
Store store = indexShard.store();
store.incRef();
try {
if (!Lucene.indexExists(store.directory()) && indexShard.state() == IndexShardState.STARTED) {
if (Lucene.indexExists(store.directory()) == false && indexShard.state() == IndexShardState.STARTED) {
return;
}
BytesStreamOutput os = new BytesStreamOutput();
PrintStream out = new PrintStream(os, false, StandardCharsets.UTF_8.name());
CheckIndex.Status status = store.checkIndex(out);
out.flush();
if (!status.clean) {
if (status.clean == false) {
logger.warn("check index [failure]\n{}", os.bytes().utf8ToString());
throw new IOException("index check failure");
}

View file

@ -126,8 +126,8 @@ public class ExceptionRetryIT extends ESIntegTestCase {
long dupCounter = 0;
boolean found_duplicate_already = false;
for (int i = 0; i < searchResponse.getHits().getHits().length; i++) {
if (!uniqueIds.add(searchResponse.getHits().getHits()[i].getId())) {
if (!found_duplicate_already) {
if (uniqueIds.add(searchResponse.getHits().getHits()[i].getId()) == false) {
if (found_duplicate_already == false) {
SearchResponse dupIdResponse = client().prepareSearch("index").setQuery(termQuery("_id",
searchResponse.getHits().getHits()[i].getId())).setExplain(true).get();
assertThat(dupIdResponse.getHits().getTotalHits().value, greaterThan(1L));

View file

@ -59,7 +59,7 @@ public class PreBuiltAnalyzerIntegrationIT extends ESIntegTestCase {
String name = preBuiltAnalyzer.name().toLowerCase(Locale.ROOT);
Version randomVersion = randomVersion(random());
if (!loadedAnalyzers.containsKey(preBuiltAnalyzer)) {
if (loadedAnalyzers.containsKey(preBuiltAnalyzer) == false) {
loadedAnalyzers.put(preBuiltAnalyzer, new ArrayList<Version>());
}
loadedAnalyzers.get(preBuiltAnalyzer).add(randomVersion);

View file

@ -383,7 +383,7 @@ public class CircuitBreakerServiceIT extends ESIntegTestCase {
// can either fail directly with an exception or the response contains exceptions (depending on client)
try {
BulkResponse response = client.bulk(bulkRequest).actionGet();
if (!response.hasFailures()) {
if (response.hasFailures() == false) {
fail("Should have thrown CircuitBreakingException");
} else {
// each item must have failed with CircuitBreakingException

View file

@ -408,7 +408,7 @@ public class IndexStatsIT extends ESIntegTestCase {
// make sure we see throttling kicking in:
boolean done = false;
long start = System.currentTimeMillis();
while (!done) {
while (done == false) {
for(int i=0; i<100; i++) {
// Provoke slowish merging by making many unique terms:
StringBuilder sb = new StringBuilder();
@ -1143,7 +1143,7 @@ public class IndexStatsIT extends ESIntegTestCase {
executionFailures.get().add(e);
latch.countDown();
}
while (!stop.get()) {
while (stop.get() == false) {
final String id = Integer.toString(idGenerator.incrementAndGet());
final IndexResponse response =
client()
@ -1171,7 +1171,7 @@ public class IndexStatsIT extends ESIntegTestCase {
final IndicesStatsRequest request = new IndicesStatsRequest();
request.all();
request.indices(new String[0]);
while (!stop.get()) {
while (stop.get() == false) {
try {
final IndicesStatsResponse response = client().admin().indices().stats(request).get();
if (response.getFailedShards() > 0) {

View file

@ -115,11 +115,11 @@ public class MinDocCountIT extends AbstractTermsTestCase {
String stringTerm;
do {
stringTerm = RandomStrings.randomAsciiOfLength(random(), 8);
} while (!stringTerms.add(stringTerm));
} while (stringTerms.add(stringTerm) == false);
long longTerm;
do {
longTerm = randomInt(cardinality * 2);
} while (!longTerms.add(longTerm));
} while (longTerms.add(longTerm) == false);
double doubleTerm = longTerm * Math.PI;
ZonedDateTime time = ZonedDateTime.of(2014, 1, ((int) longTerm % 20) + 1, 0, 0, 0, 0, ZoneOffset.UTC);

View file

@ -606,10 +606,10 @@ public class DerivativeIT extends ESIntegTestCase {
} else if (cause instanceof SearchPhaseExecutionException) {
SearchPhaseExecutionException spee = (SearchPhaseExecutionException) e;
Throwable rootCause = spee.getRootCause();
if (!(rootCause instanceof IllegalArgumentException)) {
if ((rootCause instanceof IllegalArgumentException) == false) {
throw e;
}
} else if (!(cause instanceof IllegalArgumentException)) {
} else if ((cause instanceof IllegalArgumentException) == false) {
throw e;
}
}

View file

@ -458,10 +458,10 @@ public class ExtendedStatsBucketIT extends ESIntegTestCase {
} else if (cause instanceof SearchPhaseExecutionException) {
SearchPhaseExecutionException spee = (SearchPhaseExecutionException) ex;
Throwable rootCause = spee.getRootCause();
if (!(rootCause instanceof IllegalArgumentException)) {
if ((rootCause instanceof IllegalArgumentException) == false) {
throw ex;
}
} else if (!(cause instanceof IllegalArgumentException)) {
} else if ((cause instanceof IllegalArgumentException) == false) {
throw ex;
}
}

View file

@ -416,10 +416,10 @@ public class PercentilesBucketIT extends ESIntegTestCase {
} else if (cause instanceof SearchPhaseExecutionException) {
SearchPhaseExecutionException spee = (SearchPhaseExecutionException) e;
Throwable rootCause = spee.getRootCause();
if (!(rootCause instanceof IllegalArgumentException)) {
if ((rootCause instanceof IllegalArgumentException) == false) {
throw e;
}
} else if (!(cause instanceof IllegalArgumentException)) {
} else if ((cause instanceof IllegalArgumentException) == false) {
throw e;
}
}
@ -450,10 +450,10 @@ public class PercentilesBucketIT extends ESIntegTestCase {
} else if (cause instanceof SearchPhaseExecutionException) {
SearchPhaseExecutionException spee = (SearchPhaseExecutionException) e;
Throwable rootCause = spee.getRootCause();
if (!(rootCause instanceof IllegalArgumentException)) {
if ((rootCause instanceof IllegalArgumentException) == false) {
throw e;
}
} else if (!(cause instanceof IllegalArgumentException)) {
} else if ((cause instanceof IllegalArgumentException) == false) {
throw e;
}
}

View file

@ -93,13 +93,13 @@ public class SerialDiffIT extends ESIntegTestCase {
}
private void assertValidIterators(Iterator expectedBucketIter, Iterator expectedCountsIter, Iterator expectedValuesIter) {
if (!expectedBucketIter.hasNext()) {
if (expectedBucketIter.hasNext() == false) {
fail("`expectedBucketIter` iterator ended before `actual` iterator, size mismatch");
}
if (!expectedCountsIter.hasNext()) {
if (expectedCountsIter.hasNext() == false) {
fail("`expectedCountsIter` iterator ended before `actual` iterator, size mismatch");
}
if (!expectedValuesIter.hasNext()) {
if (expectedValuesIter.hasNext() == false) {
fail("`expectedValuesIter` iterator ended before `actual` iterator, size mismatch");
}
}
@ -207,7 +207,7 @@ public class SerialDiffIT extends ESIntegTestCase {
}
// Both have values, calculate diff and replace the "empty" bucket
if (!Double.isNaN(metricValue) && !Double.isNaN(lagValue)) {
if (Double.isNaN(metricValue) == false && Double.isNaN(lagValue) == false) {
double diff = metricValue - lagValue;
values.add(diff);
} else {

View file

@ -73,7 +73,7 @@ public class SearchWhileRelocatingIT extends ESIntegTestCase {
@Override
public void run() {
try {
while (!stop.get()) {
while (stop.get() == false) {
SearchResponse sr = client().prepareSearch().setSize(numDocs).get();
if (sr.getHits().getTotalHits().value != numDocs) {
// if we did not search all shards but had no failures that is potentially fine
@ -96,7 +96,7 @@ public class SearchWhileRelocatingIT extends ESIntegTestCase {
} catch (SearchPhaseExecutionException ex) {
// it's possible that all shards fail if we have a small number of shards.
// with replicas this should not happen
if (numberOfReplicas == 1 || !ex.getMessage().contains("all shards failed")) {
if (numberOfReplicas == 1 || ex.getMessage().contains("all shards failed") == false) {
throw ex;
}
}
@ -117,7 +117,7 @@ public class SearchWhileRelocatingIT extends ESIntegTestCase {
.setWaitForNoRelocatingShards(true).setWaitForEvents(Priority.LANGUID).setTimeout("5m").get();
assertNoTimeout(resp);
// if we hit only non-critical exceptions we make sure that the post search works
if (!nonCriticalExceptions.isEmpty()) {
if (nonCriticalExceptions.isEmpty() == false) {
logger.info("non-critical exceptions: {}", nonCriticalExceptions);
for (int j = 0; j < 10; j++) {
assertHitCount(client().prepareSearch().get(), numDocs);

View file

@ -170,7 +170,7 @@ public class SearchWithRandomIOExceptionsIT extends ESIntegTestCase {
.setQuery(QueryBuilders.matchQuery("test", English.intToEnglish(docToQuery)))
.setSize(expectedResults).get();
logger.info("Successful shards: [{}] numShards: [{}]", searchResponse.getSuccessfulShards(), numShards.numPrimaries);
if (searchResponse.getSuccessfulShards() == numShards.numPrimaries && !refreshFailed) {
if (searchResponse.getSuccessfulShards() == numShards.numPrimaries && refreshFailed == false) {
assertResultsAndLogOnFailure(expectedResults, searchResponse);
}
// check match all
@ -178,14 +178,15 @@ public class SearchWithRandomIOExceptionsIT extends ESIntegTestCase {
.setSize(numCreated + numInitialDocs).addSort("_uid", SortOrder.ASC).get();
logger.info("Match all Successful shards: [{}] numShards: [{}]", searchResponse.getSuccessfulShards(),
numShards.numPrimaries);
if (searchResponse.getSuccessfulShards() == numShards.numPrimaries && !refreshFailed) {
if (searchResponse.getSuccessfulShards() == numShards.numPrimaries && refreshFailed == false) {
assertResultsAndLogOnFailure(numCreated + numInitialDocs, searchResponse);
}
} catch (SearchPhaseExecutionException ex) {
logger.info("SearchPhaseException: [{}]", ex.getMessage());
// if a scheduled refresh or flush fails all shards we see all shards failed here
if (!(expectAllShardsFailed || refreshResponse.getSuccessfulShards() == 0 ||
ex.getMessage().contains("all shards failed"))) {
if ((expectAllShardsFailed
|| refreshResponse.getSuccessfulShards() == 0
|| ex.getMessage().contains("all shards failed")) == false) {
throw ex;
}
}

View file

@ -475,7 +475,7 @@ public class QueryRescorerIT extends ESIntegTestCase {
QueryRescorerBuilder innerRescoreQuery = new QueryRescorerBuilder(matchQuery("field1", "the quick brown").boost(4.0f))
.setQueryWeight(0.5f).setRescoreQueryWeight(0.4f);
if (!"".equals(scoreModes[innerMode])) {
if ("".equals(scoreModes[innerMode]) == false) {
innerRescoreQuery.setScoreMode(QueryRescoreMode.fromString(scoreModes[innerMode]));
}
@ -497,7 +497,7 @@ public class QueryRescorerIT extends ESIntegTestCase {
QueryRescorerBuilder outerRescoreQuery = new QueryRescorerBuilder(matchQuery("field1", "the quick brown").boost(4.0f))
.setQueryWeight(0.5f).setRescoreQueryWeight(0.4f);
if (!"".equals(scoreModes[outerMode])) {
if ("".equals(scoreModes[outerMode]) == false) {
outerRescoreQuery.setScoreMode(QueryRescoreMode.fromString(scoreModes[outerMode]));
}
@ -544,7 +544,7 @@ public class QueryRescorerIT extends ESIntegTestCase {
.should(functionScoreQuery(termQuery("field1", intToEnglish[3]), weightFactorFunction(0.0f)).boostMode(REPLACE)));
rescoreQuery.setQueryWeight(primaryWeight).setRescoreQueryWeight(secondaryWeight);
if (!"".equals(scoreMode)) {
if ("".equals(scoreMode) == false) {
rescoreQuery.setScoreMode(QueryRescoreMode.fromString(scoreMode));
}

View file

@ -127,7 +127,7 @@ public class DuelScrollIT extends ESIntegTestCase {
int numMissingDocs = scaledRandomIntBetween(0, numDocs / 100);
IntHashSet missingDocs = new IntHashSet(numMissingDocs);
for (int i = 0; i < numMissingDocs; i++) {
while (!missingDocs.add(randomInt(numDocs))) {}
while (missingDocs.add(randomInt(numDocs)) == false) {}
}
for (int i = 1; i <= numDocs; i++) {

View file

@ -206,7 +206,7 @@ public class SearchAfterIT extends ESIntegTestCase {
}
for (int i = 0; i < o1.size(); i++) {
if (!(o1.get(i) instanceof Comparable)) {
if ((o1.get(i) instanceof Comparable) == false) {
throw new RuntimeException(o1.get(i).getClass() + " is not comparable");
}
Object cmp1 = o1.get(i);

View file

@ -305,7 +305,7 @@ public class FieldSortIT extends ESIntegTestCase {
assertThat(searchResponse.getHits().getAt(i).getSortValues()[0].toString(), equalTo(next.getKey().utf8ToString()));
}
}
if (!sparseBytes.isEmpty()) {
if (sparseBytes.isEmpty() == false) {
int size = between(1, sparseBytes.size());
SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery())
.setPostFilter(QueryBuilders.existsQuery("sparse_bytes")).setSize(size).addSort("sparse_bytes", SortOrder.ASC).get();

View file

@ -231,18 +231,18 @@ public class MultiGetRequest extends ActionRequest
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (!(o instanceof Item)) return false;
if ((o instanceof Item) == false) return false;
Item item = (Item) o;
if (version != item.version) return false;
if (fetchSourceContext != null ? !fetchSourceContext.equals(item.fetchSourceContext) : item.fetchSourceContext != null)
if (fetchSourceContext != null ? fetchSourceContext.equals(item.fetchSourceContext) == false : item.fetchSourceContext != null)
return false;
if (!Arrays.equals(storedFields, item.storedFields)) return false;
if (!id.equals(item.id)) return false;
if (!index.equals(item.index)) return false;
if (routing != null ? !routing.equals(item.routing) : item.routing != null) return false;
if (type != null ? !type.equals(item.type) : item.type != null) return false;
if (Arrays.equals(storedFields, item.storedFields) == false) return false;
if (id.equals(item.id) == false) return false;
if (index.equals(item.index) == false) return false;
if (routing != null ? routing.equals(item.routing) == false : item.routing != null) return false;
if (type != null ? type.equals(item.type) == false : item.type != null) return false;
if (versionType != item.versionType) return false;
return true;
@ -425,7 +425,7 @@ public class MultiGetRequest extends ActionRequest
currentFieldName = parser.currentName();
} else if (token.isValue()) {
if (INDEX.match(currentFieldName, parser.getDeprecationHandler())) {
if (!allowExplicitIndex) {
if (allowExplicitIndex == false) {
throw new IllegalArgumentException("explicit index in multi get is not allowed");
}
index = parser.text();
@ -523,7 +523,7 @@ public class MultiGetRequest extends ActionRequest
@Nullable String defaultRouting) throws IOException {
Token token;
while ((token = parser.nextToken()) != Token.END_ARRAY) {
if (!token.isValue()) {
if (token.isValue() == false) {
throw new IllegalArgumentException("ids array element should only contain ids");
}
items.add(new Item(defaultIndex, defaultType, parser.text()).storedFields(defaultFields).fetchSourceContext(defaultFetchSource)

View file

@ -132,7 +132,7 @@ public class GetPipelineResponse extends ActionResponse implements StatusToXCont
}
for (PipelineConfiguration pipeline: pipelines) {
PipelineConfiguration otherPipeline = otherPipelineMap.get(pipeline.getId());
if (!pipeline.equals(otherPipeline)) {
if (pipeline.equals(otherPipeline) == false) {
return false;
}
}

View file

@ -233,7 +233,7 @@ public class MultiSearchRequest extends ActionRequest implements CompositeIndice
for (Map.Entry<String, Object> entry : source.entrySet()) {
Object value = entry.getValue();
if ("index".equals(entry.getKey()) || "indices".equals(entry.getKey())) {
if (!allowExplicitIndex) {
if (allowExplicitIndex == false) {
throw new IllegalArgumentException("explicit index in multi search is not allowed");
}
searchRequest.indices(nodeStringArrayValue(value));

View file

@ -275,7 +275,7 @@ public final class SearchPhaseController {
ScoreDoc[] sortedDocs = reducedQueryPhase.sortedTopDocs.scoreDocs;
SearchHits hits = getHits(reducedQueryPhase, ignoreFrom, fetchResults, resultsLookup);
if (reducedQueryPhase.suggest != null) {
if (!fetchResults.isEmpty()) {
if (fetchResults.isEmpty() == false) {
int currentOffset = hits.getHits().length;
for (CompletionSuggestion suggestion : reducedQueryPhase.suggest.filter(CompletionSuggestion.class)) {
final List<CompletionSuggestion.Entry.Option> suggestionOptions = suggestion.getOptions();
@ -330,7 +330,7 @@ public final class SearchPhaseController {
numSearchHits = Math.min(sortedTopDocs.scoreDocs.length, numSearchHits);
// merge hits
List<SearchHit> hits = new ArrayList<>();
if (!fetchResults.isEmpty()) {
if (fetchResults.isEmpty() == false) {
for (int i = 0; i < numSearchHits; i++) {
ScoreDoc shardDoc = sortedTopDocs.scoreDocs[i];
SearchPhaseResult fetchResultProvider = resultsLookup.apply(shardDoc.shardIndex);
@ -659,7 +659,7 @@ public final class SearchPhaseController {
}
}
fetchHits += topDocs.topDocs.scoreDocs.length;
if (!Float.isNaN(topDocs.maxScore)) {
if (Float.isNaN(topDocs.maxScore) == false) {
maxScore = Math.max(maxScore, topDocs.maxScore);
}
if (timedOut) {

View file

@ -60,7 +60,7 @@ public final class DestructiveOperations {
* Fail if there is wildcard usage in indices and the named is required for destructive operations.
*/
public void failDestructive(String[] aliasesOrIndices) {
if (!destructiveRequiresName) {
if (destructiveRequiresName == false) {
return;
}

View file

@ -122,7 +122,7 @@ public class MultiTermVectorsRequest extends ActionRequest
}
} else if ("ids".equals(currentFieldName)) {
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (!token.isValue()) {
if (token.isValue() == false) {
throw new IllegalArgumentException("ids array element should only contain ids");
}
ids.add(parser.text());

View file

@ -179,7 +179,7 @@ public class TermVectorsFilter {
PostingsEnum docsEnum = null;
for (String fieldName : fields) {
if ((selectedFields != null) && (!selectedFields.contains(fieldName))) {
if (selectedFields != null && selectedFields.contains(fieldName) == false) {
continue;
}
@ -216,7 +216,7 @@ public class TermVectorsFilter {
// now call on docFreq
long docFreq = topLevelTermsEnum.docFreq();
if (!isAccepted(docFreq)) {
if (isAccepted(docFreq) == false) {
continue;
}

View file

@ -508,9 +508,9 @@ public class TermVectorsRequest extends SingleShardRequest<TermVectorsRequest> i
}
private void setFlag(Flag flag, boolean set) {
if (set && !flagsEnum.contains(flag)) {
if (set && flagsEnum.contains(flag) == false) {
flagsEnum.add(flag);
} else if (!set) {
} else if (set == false) {
flagsEnum.remove(flag);
assert (!flagsEnum.contains(flag));
}

View file

@ -139,7 +139,7 @@ public class TermVectorsResponse extends ActionResponse implements ToXContentObj
public Fields getFields() throws IOException {
if (hasTermVectors() && isExists()) {
if (!sourceCopied) { // make the bytes safe
if (sourceCopied == false) { // make the bytes safe
headerRef = new BytesArray(headerRef.toBytesRef(), true);
termVectors = new BytesArray(termVectors.toBytesRef(), true);
}
@ -174,7 +174,7 @@ public class TermVectorsResponse extends ActionResponse implements ToXContentObj
builder.startObject();
builder.field(FieldStrings._INDEX, index);
builder.field(FieldStrings._TYPE, type);
if (!isArtificial()) {
if (isArtificial() == false) {
builder.field(FieldStrings._ID, id);
}
builder.field(FieldStrings._VERSION, docVersion);
@ -244,7 +244,7 @@ public class TermVectorsResponse extends ActionResponse implements ToXContentObj
}
private void buildValues(XContentBuilder builder, Terms curTerms, int termFreq) throws IOException {
if (!(curTerms.hasPayloads() || curTerms.hasOffsets() || curTerms.hasPositions())) {
if ((curTerms.hasPayloads() || curTerms.hasOffsets() || curTerms.hasPositions()) == false) {
return;
}

View file

@ -152,13 +152,13 @@ public class CoordinationMetadata implements Writeable, ToXContentFragment {
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (!(o instanceof CoordinationMetadata)) return false;
if ((o instanceof CoordinationMetadata) == false) return false;
CoordinationMetadata that = (CoordinationMetadata) o;
if (term != that.term) return false;
if (!lastCommittedConfiguration.equals(that.lastCommittedConfiguration)) return false;
if (!lastAcceptedConfiguration.equals(that.lastAcceptedConfiguration)) return false;
if (lastCommittedConfiguration.equals(that.lastCommittedConfiguration) == false) return false;
if (lastAcceptedConfiguration.equals(that.lastAcceptedConfiguration) == false) return false;
return votingConfigExclusions.equals(that.votingConfigExclusions);
}

View file

@ -1016,7 +1016,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
* this method does nothing.
*/
public static boolean assertShardStats(RoutingNodes routingNodes) {
if (!Assertions.ENABLED) {
if (Assertions.ENABLED == false) {
return true;
}
int unassignedPrimaryCount = 0;

View file

@ -407,7 +407,7 @@ public class AllocationService {
private void logClusterHealthStateChange(ClusterStateHealth previousStateHealth, ClusterStateHealth newStateHealth, String reason) {
ClusterHealthStatus previousHealth = previousStateHealth.getStatus();
ClusterHealthStatus currentHealth = newStateHealth.getStatus();
if (!previousHealth.equals(currentHealth)) {
if (previousHealth.equals(currentHealth) == false) {
logger.info("Cluster health status changed from [{}] to [{}] (reason: [{}]).", previousHealth, currentHealth, reason);
}
}

View file

@ -149,7 +149,7 @@ public class ChildMemoryCircuitBreaker implements CircuitBreaker {
}
// Attempt to set the new used value, but make sure it hasn't changed
// underneath us, if it has, keep trying until we are able to set it
} while (!this.used.compareAndSet(currentUsed, newUsed));
} while (this.used.compareAndSet(currentUsed, newUsed) == false);
return newUsed;
}

View file

@ -112,7 +112,7 @@ public class Loggers {
}
public static void setLevel(Logger logger, Level level) {
if (!LogManager.ROOT_LOGGER_NAME.equals(logger.getName())) {
if (LogManager.ROOT_LOGGER_NAME.equals(logger.getName()) == false) {
Configurator.setLevel(logger.getName(), level);
} else {
final LoggerContext ctx = LoggerContext.getContext(false);
@ -136,7 +136,7 @@ public class Loggers {
final Configuration config = ctx.getConfiguration();
config.addAppender(appender);
LoggerConfig loggerConfig = config.getLoggerConfig(logger.getName());
if (!logger.getName().equals(loggerConfig.getName())) {
if (logger.getName().equals(loggerConfig.getName()) == false) {
loggerConfig = new LoggerConfig(logger.getName(), logger.getLevel(), true);
config.addLogger(logger.getName(), loggerConfig);
}
@ -148,7 +148,7 @@ public class Loggers {
final LoggerContext ctx = (LoggerContext) LogManager.getContext(false);
final Configuration config = ctx.getConfiguration();
LoggerConfig loggerConfig = config.getLoggerConfig(logger.getName());
if (!logger.getName().equals(loggerConfig.getName())) {
if (logger.getName().equals(loggerConfig.getName()) == false) {
loggerConfig = new LoggerConfig(logger.getName(), logger.getLevel(), true);
config.addLogger(logger.getName(), loggerConfig);
}

View file

@ -50,7 +50,7 @@ public class HttpPipeliningAggregator<Listener> {
if (outboundHoldingQueue.size() < maxEventsHeld) {
ArrayList<Tuple<HttpPipelinedResponse, Listener>> readyResponses = new ArrayList<>();
outboundHoldingQueue.add(new Tuple<>(response, listener));
while (!outboundHoldingQueue.isEmpty()) {
while (outboundHoldingQueue.isEmpty() == false) {
/*
* Since the response with the lowest sequence number is the top of the priority queue, we know if its sequence
* number does not match the current write sequence number then we have not processed all preceding responses yet.

View file

@ -546,7 +546,7 @@ public final class AnalysisRegistry implements Closeable {
analyzer.normalize("", ""); // check for deprecations
}
if (!analyzers.containsKey(DEFAULT_ANALYZER_NAME)) {
if (analyzers.containsKey(DEFAULT_ANALYZER_NAME) == false) {
analyzers.put(DEFAULT_ANALYZER_NAME,
produceAnalyzer(DEFAULT_ANALYZER_NAME,
new StandardAnalyzerProvider(indexSettings, null, DEFAULT_ANALYZER_NAME, Settings.Builder.EMPTY_SETTINGS),

View file

@ -163,7 +163,7 @@ public class NamedAnalyzer extends DelegatingAnalyzerWrapper {
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (!(o instanceof NamedAnalyzer)) return false;
if ((o instanceof NamedAnalyzer) == false) return false;
NamedAnalyzer that = (NamedAnalyzer) o;
return Objects.equals(name, that.name);
}

View file

@ -48,7 +48,7 @@ final class DeleteVersionValue extends VersionValue {
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
if (!super.equals(o)) return false;
if (super.equals(o) == false) return false;
DeleteVersionValue that = (DeleteVersionValue) o;

View file

@ -44,7 +44,7 @@ final class IndexVersionValue extends VersionValue {
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
if (!super.equals(o)) return false;
if (super.equals(o) == false) return false;
IndexVersionValue that = (IndexVersionValue) o;
return Objects.equals(translogLocation, that.translogLocation);
}

View file

@ -300,7 +300,7 @@ public class GetResult implements Writeable, Iterable<DocumentField>, ToXContent
XContentHelper.writeRawField(SourceFieldMapper.NAME, source, builder, params);
}
if (!documentFields.isEmpty()) {
if (documentFields.isEmpty() == false) {
builder.startObject(FIELDS);
for (DocumentField field : documentFields.values()) {
field.toXContent(builder, params);

View file

@ -128,7 +128,7 @@ public final class ShardGetService extends AbstractIndexShardComponent {
*/
public GetResult get(Engine.GetResult engineGetResult, String id, String type,
String[] fields, FetchSourceContext fetchSourceContext) {
if (!engineGetResult.exists()) {
if (engineGetResult.exists() == false) {
return new GetResult(shardId.getIndexName(), type, id, UNASSIGNED_SEQ_NO, UNASSIGNED_PRIMARY_TERM, -1, false, null, null, null);
}
@ -280,7 +280,7 @@ public final class ShardGetService extends AbstractIndexShardComponent {
}
// put stored fields into result objects
if (!fieldVisitor.fields().isEmpty()) {
if (fieldVisitor.fields().isEmpty() == false) {
fieldVisitor.postProcess(mapperService::fieldType,
mapperService.documentMapper() == null ? null : mapperService.documentMapper().type());
documentFields = new HashMap<>();

View file

@ -422,7 +422,7 @@ public class CompletionFieldMapper extends FieldMapper {
while ((token = parser.nextToken()) != Token.END_OBJECT) {
if (token == Token.FIELD_NAME) {
currentFieldName = parser.currentName();
if (!ALLOWED_CONTENT_FIELD_NAMES.contains(currentFieldName)) {
if (ALLOWED_CONTENT_FIELD_NAMES.contains(currentFieldName) == false) {
throw new IllegalArgumentException("unknown field name [" + currentFieldName
+ "], must be one of " + ALLOWED_CONTENT_FIELD_NAMES);
}

View file

@ -140,7 +140,7 @@ public class DocumentMapperParser {
}
public static void checkNoRemainingFields(Map<?, ?> fieldNodeMap, String message) {
if (!fieldNodeMap.isEmpty()) {
if (fieldNodeMap.isEmpty() == false) {
throw new MapperParsingException(message + getRemainingFields(fieldNodeMap));
}
}

View file

@ -457,7 +457,7 @@ final class DocumentParser {
if (nested.isIncludeInRoot()) {
ParseContext.Document rootDoc = context.rootDoc();
// don't add it twice, if its included in parent, and we are handling the master doc...
if (!nested.isIncludeInParent() || parentDoc != rootDoc) {
if (nested.isIncludeInParent() == false || parentDoc != rootDoc) {
addFields(nestedDoc, rootDoc);
}
}
@ -664,7 +664,7 @@ final class DocumentParser {
/** Creates instances of the fields that the current field should be copied to */
private static void parseCopyFields(ParseContext context, List<String> copyToFields) throws IOException {
if (!context.isWithinCopyTo() && copyToFields.isEmpty() == false) {
if (context.isWithinCopyTo() == false && copyToFields.isEmpty() == false) {
context = context.createCopyToContext();
for (String field : copyToFields) {
// In case of a hierarchy of nested documents, we need to figure out

View file

@ -68,7 +68,7 @@ public final class FieldAliasMapper extends Mapper {
@Override
public Mapper merge(Mapper mergeWith) {
if (!(mergeWith instanceof FieldAliasMapper)) {
if ((mergeWith instanceof FieldAliasMapper) == false) {
throw new IllegalArgumentException("Cannot merge a field alias mapping ["
+ name() + "] with a mapping that is not for a field alias.");
}
@ -105,7 +105,7 @@ public final class FieldAliasMapper extends Mapper {
String aliasScope = mappers.getNestedScope(name);
String pathScope = mappers.getNestedScope(path);
if (!Objects.equals(aliasScope, pathScope)) {
if (Objects.equals(aliasScope, pathScope) == false) {
StringBuilder message = new StringBuilder("Invalid [path] value [" + path + "] for field alias [" +
name + "]: an alias must have the same nested scope as its target. ");
message.append(aliasScope == null

View file

@ -407,7 +407,7 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (!mappers.isEmpty()) {
if (mappers.isEmpty() == false) {
// sort the mappers so we get consistent serialization format
List<FieldMapper> sortedMappers = new ArrayList<>(mappers.values());
sortedMappers.sort(Comparator.comparing(FieldMapper::name));
@ -439,7 +439,7 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
}
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (!copyToFields.isEmpty()) {
if (copyToFields.isEmpty() == false) {
builder.startArray("copy_to");
for (String field : copyToFields) {
builder.value(field);

View file

@ -226,7 +226,7 @@ public class NumberFieldMapper extends FieldMapper {
}
private void validateParsed(float value) {
if (!Float.isFinite(HalfFloatPoint.sortableShortToHalfFloat(HalfFloatPoint.halfFloatToSortableShort(value)))) {
if (Float.isFinite(HalfFloatPoint.sortableShortToHalfFloat(HalfFloatPoint.halfFloatToSortableShort(value))) == false) {
throw new IllegalArgumentException("[half_float] supports only finite values, but got [" + value + "]");
}
}
@ -321,7 +321,7 @@ public class NumberFieldMapper extends FieldMapper {
}
private void validateParsed(float value) {
if (!Float.isFinite(value)) {
if (Float.isFinite(value) == false) {
throw new IllegalArgumentException("[float] supports only finite values, but got [" + value + "]");
}
}
@ -392,7 +392,7 @@ public class NumberFieldMapper extends FieldMapper {
}
private void validateParsed(double value) {
if (!Double.isFinite(value)) {
if (Double.isFinite(value) == false) {
throw new IllegalArgumentException("[double] supports only finite values, but got [" + value + "]");
}
}
@ -405,7 +405,7 @@ public class NumberFieldMapper extends FieldMapper {
if (doubleValue < Byte.MIN_VALUE || doubleValue > Byte.MAX_VALUE) {
throw new IllegalArgumentException("Value [" + value + "] is out of range for a byte");
}
if (!coerce && doubleValue % 1 != 0) {
if (coerce == false && doubleValue % 1 != 0) {
throw new IllegalArgumentException("Value [" + value + "] has a decimal part");
}
@ -466,7 +466,7 @@ public class NumberFieldMapper extends FieldMapper {
if (doubleValue < Short.MIN_VALUE || doubleValue > Short.MAX_VALUE) {
throw new IllegalArgumentException("Value [" + value + "] is out of range for a short");
}
if (!coerce && doubleValue % 1 != 0) {
if (coerce == false && doubleValue % 1 != 0) {
throw new IllegalArgumentException("Value [" + value + "] has a decimal part");
}
@ -523,7 +523,7 @@ public class NumberFieldMapper extends FieldMapper {
if (doubleValue < Integer.MIN_VALUE || doubleValue > Integer.MAX_VALUE) {
throw new IllegalArgumentException("Value [" + value + "] is out of range for an integer");
}
if (!coerce && doubleValue % 1 != 0) {
if (coerce == false && doubleValue % 1 != 0) {
throw new IllegalArgumentException("Value [" + value + "] has a decimal part");
}
@ -559,7 +559,7 @@ public class NumberFieldMapper extends FieldMapper {
int upTo = 0;
for (Object value : values) {
if (!hasDecimalPart(value)) {
if (hasDecimalPart(value) == false) {
v[upTo++] = parse(value, true);
}
}
@ -664,7 +664,7 @@ public class NumberFieldMapper extends FieldMapper {
int upTo = 0;
for (Object value : values) {
if (!hasDecimalPart(value)) {
if (hasDecimalPart(value) == false) {
v[upTo++] = parse(value, true);
}
}
@ -816,7 +816,7 @@ public class NumberFieldMapper extends FieldMapper {
if (doubleValue < Long.MIN_VALUE || doubleValue > Long.MAX_VALUE) {
throw new IllegalArgumentException("Value [" + value + "] is out of range for a long");
}
if (!coerce && doubleValue % 1 != 0) {
if (coerce == false && doubleValue % 1 != 0) {
throw new IllegalArgumentException("Value [" + value + "] has a decimal part");
}

View file

@ -100,7 +100,7 @@ public class ObjectMapper extends Mapper implements Cloneable {
public void merge(Nested mergeWith, MergeReason reason) {
if (isNested()) {
if (!mergeWith.isNested()) {
if (mergeWith.isNested() == false) {
throw new IllegalArgumentException("cannot change object mapping from nested to non-nested");
}
} else {
@ -247,7 +247,7 @@ public class ObjectMapper extends Mapper implements Cloneable {
} else if (fieldName.equals("properties")) {
if (fieldNode instanceof Collection && ((Collection) fieldNode).isEmpty()) {
// nothing to do here, empty (to support "properties: []" case)
} else if (!(fieldNode instanceof Map)) {
} else if ((fieldNode instanceof Map) == false) {
throw new ElasticsearchParseException("properties must be a map type");
} else {
parseProperties(builder, (Map<String, Object>) fieldNode, parserContext);
@ -478,7 +478,7 @@ public class ObjectMapper extends Mapper implements Cloneable {
}
public ObjectMapper merge(Mapper mergeWith, MergeReason reason) {
if (!(mergeWith instanceof ObjectMapper)) {
if ((mergeWith instanceof ObjectMapper) == false) {
throw new IllegalArgumentException("can't merge a non object mapping [" + mergeWith.name() + "] with an object mapping");
}
ObjectMapper mergeWithObject = (ObjectMapper) mergeWith;
@ -569,7 +569,7 @@ public class ObjectMapper extends Mapper implements Cloneable {
int count = 0;
for (Mapper mapper : sortedMappers) {
if (!(mapper instanceof MetadataFieldMapper)) {
if ((mapper instanceof MetadataFieldMapper) == false) {
if (count++ == 0) {
builder.startObject("properties");
}

View file

@ -678,11 +678,11 @@ public abstract class ParseContext {
* @return null if no external value has been set or the value
*/
public final <T> T parseExternalValue(Class<T> clazz) {
if (!externalValueSet() || externalValue() == null) {
if (externalValueSet() == false || externalValue() == null) {
return null;
}
if (!clazz.isInstance(externalValue())) {
if (clazz.isInstance(externalValue()) == false) {
throw new IllegalArgumentException("illegal external value class ["
+ externalValue().getClass().getName() + "]. Should be " + clazz.getName());
}

View file

@ -206,7 +206,7 @@ public class RangeFieldMapper extends FieldMapper {
@SuppressWarnings("unchecked")
protected Object parseSourceValue(Object value) {
RangeType rangeType = rangeType();
if (!(value instanceof Map)) {
if ((value instanceof Map) == false) {
assert rangeType == RangeType.IP;
Tuple<InetAddress, Integer> ipRange = InetAddresses.parseCidr(value.toString());
return InetAddresses.toCidrString(ipRange.v1(), ipRange.v2());

View file

@ -123,7 +123,7 @@ public class TypeParsers {
throw new MapperParsingException("Field name [" + multiFieldName + "] which is a multi field of [" + name + "] cannot" +
" contain '.'");
}
if (!(multiFieldEntry.getValue() instanceof Map)) {
if ((multiFieldEntry.getValue() instanceof Map) == false) {
throw new MapperParsingException("illegal field [" + multiFieldName + "], only fields can be specified inside fields");
}
@SuppressWarnings("unchecked")

View file

@ -409,7 +409,7 @@ public abstract class AbstractGeometryQueryBuilder<QB extends AbstractGeometryQu
@Override
public void onResponse(GetResponse response) {
try {
if (!response.isExists()) {
if (response.isExists() == false) {
throw new IllegalArgumentException("Shape with ID [" + getRequest.id() + "] in type [" + getRequest.type()
+ "] not found");
}

View file

@ -174,10 +174,10 @@ public class BoostingQueryBuilder extends AbstractQueryBuilder<BoostingQueryBuil
}
}
if (!positiveQueryFound) {
if (positiveQueryFound == false) {
throw new ParsingException(parser.getTokenLocation(), "[boosting] query requires 'positive' query to be set'");
}
if (!negativeQueryFound) {
if (negativeQueryFound == false) {
throw new ParsingException(parser.getTokenLocation(), "[boosting] query requires 'negative' query to be set'");
}
if (negativeBoost < 0) {

View file

@ -117,7 +117,7 @@ public class ConstantScoreQueryBuilder extends AbstractQueryBuilder<ConstantScor
throw new ParsingException(parser.getTokenLocation(), "unexpected token [" + token + "]");
}
}
if (!queryFound) {
if (queryFound == false) {
throw new ParsingException(parser.getTokenLocation(), "[constant_score] requires a 'filter' element");
}

View file

@ -165,7 +165,7 @@ public class DisMaxQueryBuilder extends AbstractQueryBuilder<DisMaxQueryBuilder>
}
}
if (!queriesFound) {
if (queriesFound == false) {
throw new ParsingException(parser.getTokenLocation(), "[dis_max] requires 'queries' field with at least one clause");
}

View file

@ -311,7 +311,7 @@ public class GeoBoundingBoxQueryBuilder extends AbstractQueryBuilder<GeoBounding
throw new QueryShardException(context, "failed to find geo field [" + fieldName + "]");
}
}
if (!(fieldType instanceof GeoShapeQueryable)) {
if ((fieldType instanceof GeoShapeQueryable) == false) {
throw new QueryShardException(context,
"Field [" + fieldName + "] is of unsupported type [" + fieldType.typeName() + "] for [" + NAME + "] query");
}

View file

@ -236,7 +236,7 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
}
}
if (!(fieldType instanceof GeoShapeQueryable)) {
if ((fieldType instanceof GeoShapeQueryable) == false) {
throw new QueryShardException(context,
"Field [" + fieldName + "] is of unsupported type [" + fieldType.typeName() + "] for [" + NAME + "] query");
}

View file

@ -82,7 +82,7 @@ public class GeoPolygonQueryBuilder extends AbstractQueryBuilder<GeoPolygonQuery
}
this.fieldName = fieldName;
this.shell = new ArrayList<>(points);
if (!shell.get(shell.size() - 1).equals(shell.get(0))) {
if (shell.get(shell.size() - 1).equals(shell.get(0)) == false) {
shell.add(shell.get(0));
}
}
@ -161,7 +161,7 @@ public class GeoPolygonQueryBuilder extends AbstractQueryBuilder<GeoPolygonQuery
throw new QueryShardException(context, "failed to find geo_point field [" + fieldName + "]");
}
}
if (!(fieldType instanceof GeoPointFieldType)) {
if ((fieldType instanceof GeoPointFieldType) == false) {
throw new QueryShardException(context, "field [" + fieldName + "] is not a geo_point field");
}
@ -173,13 +173,13 @@ public class GeoPolygonQueryBuilder extends AbstractQueryBuilder<GeoPolygonQuery
// validation was not available prior to 2.x, so to support bwc
// percolation queries we only ignore_malformed on 2.x created indexes
if (!GeoValidationMethod.isIgnoreMalformed(validationMethod)) {
if (GeoValidationMethod.isIgnoreMalformed(validationMethod) == false) {
for (GeoPoint point : shell) {
if (!GeoUtils.isValidLatitude(point.lat())) {
if (GeoUtils.isValidLatitude(point.lat()) == false) {
throw new QueryShardException(context, "illegal latitude value [{}] for [{}]", point.lat(),
GeoPolygonQueryBuilder.NAME);
}
if (!GeoUtils.isValidLongitude(point.lon())) {
if (GeoUtils.isValidLongitude(point.lon()) == false) {
throw new QueryShardException(context, "illegal longitude value [{}] for [{}]", point.lon(),
GeoPolygonQueryBuilder.NAME);
}

View file

@ -510,7 +510,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (!(o instanceof Item)) return false;
if ((o instanceof Item) == false) return false;
Item other = (Item) o;
return Objects.equals(index, other.index)
&& Objects.equals(type, other.type)
@ -1124,7 +1124,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
boolQuery.add(mltQuery, BooleanClause.Occur.SHOULD);
// exclude the items from the search
if (!include) {
if (include == false) {
handleExclude(boolQuery, likeItems, context);
}
return boolQuery.build();
@ -1166,7 +1166,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
continue;
}
TermVectorsResponse getResponse = response.getResponse();
if (!getResponse.isExists()) {
if (getResponse.isExists() == false) {
continue;
}
likeFields.add(getResponse.getFields());
@ -1195,7 +1195,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
}
ids.add(item.id());
}
if (!ids.isEmpty()) {
if (ids.isEmpty() == false) {
Query query = idField.termsQuery(ids, context);
boolQuery.add(query, BooleanClause.Occur.MUST_NOT);
}

View file

@ -281,7 +281,7 @@ public class NestedQueryBuilder extends AbstractQueryBuilder<NestedQueryBuilder>
throw new IllegalStateException("[" + NAME + "] failed to find nested object under path [" + path + "]");
}
}
if (!nestedObjectMapper.nested().isNested()) {
if (nestedObjectMapper.nested().isNested() == false) {
throw new IllegalStateException("[" + NAME + "] nested object under path [" + path + "] is not of nested type");
}
final BitSetProducer parentFilter;

View file

@ -98,7 +98,7 @@ public class RangeQueryBuilder extends AbstractQueryBuilder<RangeQueryBuilder> i
String relationString = in.readOptionalString();
if (relationString != null) {
relation = ShapeRelation.getRelationByName(relationString);
if (relation != null && !isRelationAllowed(relation)) {
if (relation != null && isRelationAllowed(relation) == false) {
throw new IllegalArgumentException(
"[range] query does not support relation [" + relationString + "]");
}
@ -310,7 +310,7 @@ public class RangeQueryBuilder extends AbstractQueryBuilder<RangeQueryBuilder> i
if (this.relation == null) {
throw new IllegalArgumentException(relation + " is not a valid relation");
}
if (!isRelationAllowed(this.relation)) {
if (isRelationAllowed(this.relation) == false) {
throw new IllegalArgumentException("[range] query does not support relation [" + relation + "]");
}
return this;

View file

@ -53,7 +53,7 @@ public enum SimpleQueryStringFlag {
}
static int resolveFlags(String flags) {
if (!Strings.hasLength(flags)) {
if (Strings.hasLength(flags) == false) {
return ALL.value();
}
int magic = NONE.value();

View file

@ -213,7 +213,7 @@ public class SpanNearQueryBuilder extends AbstractQueryBuilder<SpanNearQueryBuil
SpanQueryBuilder queryBuilder = clauses.get(0);
boolean isGap = queryBuilder instanceof SpanGapQueryBuilder;
Query query = null;
if (!isGap) {
if (isGap == false) {
query = queryBuilder.toQuery(context);
assert query instanceof SpanQuery;
}
@ -249,7 +249,7 @@ public class SpanNearQueryBuilder extends AbstractQueryBuilder<SpanNearQueryBuil
String fieldName = ((SpanGapQueryBuilder) queryBuilder).fieldName();
String spanGapFieldName = queryFieldName(context, fieldName);
if (!spanNearFieldName.equals(spanGapFieldName)) {
if (spanNearFieldName.equals(spanGapFieldName) == false) {
throw new IllegalArgumentException("[span_near] clauses must have same field");
}
int gap = ((SpanGapQueryBuilder) queryBuilder).width();

View file

@ -122,7 +122,7 @@ public class WrapperQueryBuilder extends AbstractQueryBuilder<WrapperQueryBuilde
throw new ParsingException(parser.getTokenLocation(), "[wrapper] query malformed");
}
String fieldName = parser.currentName();
if (! QUERY_FIELD.match(fieldName, parser.getDeprecationHandler())) {
if (QUERY_FIELD.match(fieldName, parser.getDeprecationHandler()) == false) {
throw new ParsingException(parser.getTokenLocation(), "[wrapper] query malformed, expected `query` but was " + fieldName);
}
parser.nextToken();

View file

@ -141,7 +141,7 @@ public class BulkByScrollTask extends CancellableTask {
* a leader task.
*/
public LeaderBulkByScrollTaskState getLeaderState() {
if (!isLeader()) {
if (isLeader() == false) {
throw new IllegalStateException("This task is not set to be a leader for other slice subtasks");
}
return leaderState;
@ -178,7 +178,7 @@ public class BulkByScrollTask extends CancellableTask {
* worker task.
*/
public WorkerBulkByScrollTaskState getWorkerState() {
if (!isWorker()) {
if (isWorker() == false) {
throw new IllegalStateException("This task is not set to be a worker");
}
return workerState;

View file

@ -252,7 +252,7 @@ public class RemoteInfo implements Writeable, ToXContentObject {
if (query == null) {
return BytesReference.bytes(matchAllQuery().toXContent(builder, ToXContent.EMPTY_PARAMS));
}
if (!(query instanceof Map)) {
if ((query instanceof Map) == false) {
throw new IllegalArgumentException("Expected [query] to be an object but was [" + query + "]");
}
@SuppressWarnings("unchecked")

View file

@ -609,7 +609,7 @@ public class MatchQuery {
OffsetAttribute offsetAtt = stream.addAttribute(OffsetAttribute.class);
stream.reset();
if (!stream.incrementToken()) {
if (stream.incrementToken() == false) {
throw new AssertionError();
}
final Term term = new Term(field, termAtt.getBytesRef());

View file

@ -128,7 +128,7 @@ public class MultiMatchQuery extends MatchQuery {
MappedFieldType fieldType = context.getFieldType(name);
if (fieldType != null) {
Analyzer actualAnalyzer = getAnalyzer(fieldType, type == MultiMatchQueryBuilder.Type.PHRASE);
if (!groups.containsKey(actualAnalyzer)) {
if (groups.containsKey(actualAnalyzer) == false) {
groups.put(actualAnalyzer, new ArrayList<>());
}
float boost = entry.getValue() == null ? 1.0f : entry.getValue();

View file

@ -562,7 +562,9 @@ public class QueryStringQueryParser extends XQueryParser {
while (true) {
try {
if (!source.incrementToken()) break;
if (source.incrementToken() == false) {
break;
}
} catch (IOException e) {
break;
}

View file

@ -478,7 +478,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
currentRouting = this.shardRouting;
assert currentRouting != null;
if (!newRouting.shardId().equals(shardId())) {
if (newRouting.shardId().equals(shardId()) == false) {
throw new IllegalArgumentException("Trying to set a routing entry with shardId " +
newRouting.shardId() + " on a shard with shardId " + shardId());
}
@ -2670,7 +2670,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
private void doCheckIndex() throws IOException {
long timeNS = System.nanoTime();
if (!Lucene.indexExists(store.directory())) {
if (Lucene.indexExists(store.directory()) == false) {
return;
}
BytesStreamOutput os = new BytesStreamOutput();
@ -2699,7 +2699,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
// full checkindex
final CheckIndex.Status status = store.checkIndex(out);
out.flush();
if (!status.clean) {
if (status.clean == false) {
if (state == IndexShardState.CLOSED) {
// ignore if closed....
return;

View file

@ -168,7 +168,7 @@ final class IndexShardOperationPermits implements Closeable {
queuedActions = Collections.emptyList();
}
}
if (!queuedActions.isEmpty()) {
if (queuedActions.isEmpty() == false) {
/*
* Try acquiring permits on fresh thread (for two reasons):
* - blockOperations can be called on a recovery thread which can be expected to be interrupted when recovery is cancelled;

View file

@ -96,7 +96,7 @@ final class InternalIndexingStats implements IndexingOperationListener {
@Override
public void postIndex(ShardId shardId, Engine.Index index, Exception ex) {
if (!index.origin().isRecovery()) {
if (index.origin().isRecovery() == false) {
totalStats.indexCurrent.dec();
typeStats(index.type()).indexCurrent.dec();
totalStats.indexFailed.inc();
@ -106,7 +106,7 @@ final class InternalIndexingStats implements IndexingOperationListener {
@Override
public Engine.Delete preDelete(ShardId shardId, Engine.Delete delete) {
if (!delete.origin().isRecovery()) {
if (delete.origin().isRecovery() == false) {
totalStats.deleteCurrent.inc();
typeStats(delete.type()).deleteCurrent.inc();
}
@ -118,7 +118,7 @@ final class InternalIndexingStats implements IndexingOperationListener {
public void postDelete(ShardId shardId, Engine.Delete delete, Engine.DeleteResult result) {
switch (result.getResultType()) {
case SUCCESS:
if (!delete.origin().isRecovery()) {
if (delete.origin().isRecovery() == false) {
long took = result.getTook();
totalStats.deleteMetric.inc(took);
totalStats.deleteCurrent.dec();
@ -137,7 +137,7 @@ final class InternalIndexingStats implements IndexingOperationListener {
@Override
public void postDelete(ShardId shardId, Engine.Delete delete, Exception ex) {
if (!delete.origin().isRecovery()) {
if (delete.origin().isRecovery() == false) {
totalStats.deleteCurrent.dec();
typeStats(delete.type()).deleteCurrent.dec();
}

View file

@ -265,7 +265,7 @@ public class PrimaryReplicaSyncer {
}
final long trimmedAboveSeqNo = firstMessage.get() ? maxSeqNo : SequenceNumbers.UNASSIGNED_SEQ_NO;
// have to send sync request even in case of there are no operations to sync - have to sync trimmedAboveSeqNo at least
if (!operations.isEmpty() || trimmedAboveSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) {
if (operations.isEmpty() == false || trimmedAboveSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) {
task.setPhase("sending_ops");
ResyncReplicationRequest request =
new ResyncReplicationRequest(shardId, trimmedAboveSeqNo, maxSeenAutoIdTimestamp, operations.toArray(EMPTY_ARRAY));

View file

@ -122,8 +122,8 @@ public class ReplicationGroup {
ReplicationGroup that = (ReplicationGroup) o;
if (!routingTable.equals(that.routingTable)) return false;
if (!inSyncAllocationIds.equals(that.inSyncAllocationIds)) return false;
if (routingTable.equals(that.routingTable) == false) return false;
if (inSyncAllocationIds.equals(that.inSyncAllocationIds) == false) return false;
return trackedAllocationIds.equals(that.trackedAllocationIds);
}

View file

@ -538,7 +538,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
// throw exception if the file is corrupt
String checksum = Store.digestToString(CodecUtil.checksumEntireFile(input));
// throw exception if metadata is inconsistent
if (!checksum.equals(md.checksum())) {
if (checksum.equals(md.checksum()) == false) {
throw new CorruptIndexException("inconsistent metadata: lucene checksum=" + checksum +
", metadata checksum=" + md.checksum(), input);
}
@ -1206,7 +1206,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
private void readAndCompareChecksum() throws IOException {
actualChecksum = digestToString(getChecksum());
if (!metadata.checksum().equals(actualChecksum)) {
if (metadata.checksum().equals(actualChecksum) == false) {
throw new CorruptIndexException("checksum failed (hardware problem?) : expected=" + metadata.checksum() +
" actual=" + actualChecksum +
" (resource=" + metadata.toString() + ")", "VerifyingIndexOutput(" + metadata.name() + ")");
@ -1376,7 +1376,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
*/
public void markStoreCorrupted(IOException exception) throws IOException {
ensureOpen();
if (!isMarkedCorrupted()) {
if (isMarkedCorrupted() == false) {
final String corruptionMarkerName = CORRUPTED_MARKER_NAME_PREFIX + UUIDs.randomBase64UUID();
try (IndexOutput output = this.directory().createOutput(corruptionMarkerName, IOContext.DEFAULT)) {
CodecUtil.writeHeader(output, CODEC, CORRUPTED_MARKER_CODEC_VERSION);

View file

@ -192,7 +192,7 @@ public class TermVectorsService {
Set<String> validFields = new HashSet<>();
for (String field : selectedFields) {
MappedFieldType fieldType = indexShard.mapperService().fieldType(field);
if (!isValidField(fieldType)) {
if (isValidField(fieldType) == false) {
continue;
}
// already retrieved, only if the analyzer hasn't been overridden at the field
@ -296,10 +296,10 @@ public class TermVectorsService {
Collection<DocumentField> documentFields = new HashSet<>();
for (IndexableField field : doc.getFields()) {
MappedFieldType fieldType = indexShard.mapperService().fieldType(field.name());
if (!isValidField(fieldType)) {
if (isValidField(fieldType) == false) {
continue;
}
if (request.selectedFields() != null && !request.selectedFields().contains(field.name())) {
if (request.selectedFields() != null && request.selectedFields().contains(field.name()) == false) {
continue;
}
if (seenFields.contains(field.name())) {

View file

@ -409,7 +409,7 @@ public class StrictISODateTimeFormat {
boolean minute = fields.remove(DateTimeFieldType.minuteOfHour());
boolean second = fields.remove(DateTimeFieldType.secondOfMinute());
boolean milli = fields.remove(DateTimeFieldType.millisOfSecond());
if (!hour && !minute && !second && !milli) {
if (hour == false && minute == false && second == false && milli == false) {
return;
}
if (hour || minute || second || milli) {
@ -420,13 +420,13 @@ public class StrictISODateTimeFormat {
bld.appendLiteral('T');
}
}
if (hour && minute && second || (hour && !second && !milli)) {
if (hour && minute && second || (hour && second == false && milli == false)) {
// OK - HMSm/HMS/HM/H - valid in combination with date
} else {
if (strictISO && datePresent) {
throw new IllegalArgumentException("No valid ISO8601 format for fields because Time was truncated: " + fields);
}
if (!hour && (minute && second || (minute && !milli) || second)) {
if (hour == false && (minute && second || (minute && milli == false) || second)) {
// OK - MSm/MS/M/Sm/S - valid ISO formats
} else {
if (strictISO) {

View file

@ -104,7 +104,7 @@ public class TransportActionFilterChainTests extends ESTestCase {
if (testFilter.callback == RequestOperation.LISTENER_FAILURE) {
errorExpected = true;
}
if (!(testFilter.callback == RequestOperation.CONTINUE_PROCESSING) ) {
if (testFilter.callback != RequestOperation.CONTINUE_PROCESSING) {
break;
}
}
@ -130,7 +130,7 @@ public class TransportActionFilterChainTests extends ESTestCase {
for (ActionFilter filter : testFiltersByLastExecution) {
RequestTestFilter testFilter = (RequestTestFilter) filter;
finalTestFilters.add(testFilter);
if (!(testFilter.callback == RequestOperation.CONTINUE_PROCESSING) ) {
if (testFilter.callback != RequestOperation.CONTINUE_PROCESSING) {
break;
}
}
@ -186,7 +186,7 @@ public class TransportActionFilterChainTests extends ESTestCase {
}
}, latch));
if (!latch.await(10, TimeUnit.SECONDS)) {
if (latch.await(10, TimeUnit.SECONDS) == false) {
fail("timeout waiting for the filter to notify the listener as many times as expected");
}

View file

@ -203,7 +203,7 @@ public class GetTermVectorsTests extends ESSingleNodeTestCase {
String resultString = "";
Map<String, Integer> payloadCounter = new HashMap<>();
for (String token : tokens) {
if (!payloadCounter.containsKey(token)) {
if (payloadCounter.containsKey(token) == false) {
payloadCounter.putIfAbsent(token, 0);
} else {
payloadCounter.put(token, payloadCounter.get(token) + 1);

View file

@ -201,7 +201,7 @@ public abstract class AbstractClientHeadersTestCase extends ESTestCase {
public Throwable unwrap(Throwable t, Class<? extends Throwable> exceptionType) {
int counter = 0;
Throwable result = t;
while (!exceptionType.isInstance(result)) {
while (exceptionType.isInstance(result) == false) {
if (result.getCause() == null) {
return null;
}

View file

@ -298,7 +298,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
nodeBuilder.add(newNode("node" + Integer.toString(i)));
}
clusterState = ClusterState.builder(clusterState).nodes(nodeBuilder).build();
while (!clusterState.routingTable().shardsWithState(UNASSIGNED).isEmpty()) {
while (clusterState.routingTable().shardsWithState(UNASSIGNED).isEmpty() == false) {
clusterState = startInitializingShardsAndReroute(strategy, clusterState);
}
@ -328,7 +328,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
}
for (String failedNode : failedNodes) {
if (!routingNodes.node(failedNode).isEmpty()) {
if (routingNodes.node(failedNode).isEmpty() == false) {
fail("shard was re-assigned to failed node " + failedNode);
}
}

View file

@ -444,7 +444,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
mutableShardRoutings = routingNodes.shardsWithState(ShardRoutingState.INITIALIZING);
for (ShardRouting r : mutableShardRoutings) {
if (!r.primary()) {
if (r.primary() == false) {
ShardRouting primary = routingNodes.activePrimary(r.shardId());
assertThat(primary, notNullValue());
String fromId = primary.currentNodeId();

View file

@ -452,7 +452,7 @@ public class MasterServiceTests extends ESTestCase {
}
public void execute() {
if (!state.compareAndSet(false, true)) {
if (state.compareAndSet(false, true) == false) {
throw new IllegalStateException();
} else {
counter.incrementAndGet();

View file

@ -758,7 +758,7 @@ public class CacheTests extends ESTestCase {
Set<Long> ids = threads.stream().map(t -> t.getId()).collect(Collectors.toSet());
ThreadMXBean mxBean = ManagementFactory.getThreadMXBean();
long[] deadlockedThreads = mxBean.findDeadlockedThreads();
if (!deadlock.get() && deadlockedThreads != null) {
if (deadlock.get() == false && deadlockedThreads != null) {
for (long deadlockedThread : deadlockedThreads) {
// ensure that we detected deadlock on our threads
if (ids.contains(deadlockedThread)) {

View file

@ -141,7 +141,7 @@ public class FreqTermsEnumTests extends ESTestCase {
for (int docId = 0; docId < reader.maxDoc(); docId++) {
Document doc = reader.document(docId);
addFreqs(doc, referenceAll);
if (!deletedIds.contains(doc.getField("id").stringValue())) {
if (deletedIds.contains(doc.getField("id").stringValue()) == false) {
addFreqs(doc, referenceNotDeleted);
if (randomBoolean()) {
filterTerms.add(new BytesRef(doc.getField("id").stringValue()));
@ -157,7 +157,7 @@ public class FreqTermsEnumTests extends ESTestCase {
for (IndexableField field : doc.getFields("field")) {
String term = field.stringValue();
FreqHolder freqHolder = reference.get(term);
if (!addedDocFreq.contains(term)) {
if (addedDocFreq.contains(term) == false) {
freqHolder.docFreq++;
addedDocFreq.add(term);
}
@ -204,7 +204,7 @@ public class FreqTermsEnumTests extends ESTestCase {
Collections.shuffle(terms, random());
for (String term : terms) {
if (!termsEnum.seekExact(new BytesRef(term))) {
if (termsEnum.seekExact(new BytesRef(term)) == false) {
assertThat("term : " + term, reference.get(term).docFreq, is(0));
continue;
}

View file

@ -86,7 +86,7 @@ public class CancellableThreadsTests extends ESTestCase {
readyForCancel.countDown();
try {
if (plan.busySpin) {
while (!Thread.currentThread().isInterrupted()) {
while (Thread.currentThread().isInterrupted() == false) {
}
} else {
Thread.sleep(50000);

View file

@ -59,7 +59,7 @@ public class PrioritizedExecutorsTests extends ESTestCase {
}
Priority prevPriority = null;
while (!queue.isEmpty()) {
while (queue.isEmpty() == false) {
if (prevPriority == null) {
prevPriority = queue.poll();
} else {

View file

@ -118,7 +118,7 @@ public class SeedHostsResolverTests extends ESTestCase {
logger.info("shutting down...");
// JDK stack is broken, it does not iterate in the expected order (http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4475301)
final List<Closeable> reverse = new ArrayList<>();
while (!closeables.isEmpty()) {
while (closeables.isEmpty() == false) {
reverse.add(closeables.pop());
}
IOUtils.close(reverse);

View file

@ -327,7 +327,7 @@ public abstract class AbstractStringFieldDataTestCase extends AbstractFieldDataI
final String docValue = searcher.doc(topDocs.scoreDocs[i].doc).get("value");
if (first && docValue == null) {
assertNull(previousValue);
} else if (!first && docValue != null) {
} else if (first == false && docValue != null) {
assertNotNull(previousValue);
}
final BytesRef value = docValue == null ? null : new BytesRef(docValue);

View file

@ -110,7 +110,7 @@ public class MultiOrdinalsTests extends ESTestCase {
if (docId == ordAndId.id) {
docOrds.add(ordAndId.ord);
} else {
if (!docOrds.isEmpty()) {
if (docOrds.isEmpty() == false) {
assertTrue(singleOrds.advanceExact(docId));
assertThat((long) singleOrds.ordValue(), equalTo(docOrds.get(0)));

View file

@ -89,7 +89,7 @@ public class DocumentFieldTests extends ESTestCase {
mutations.add(() -> new DocumentField(documentField.getName(), randomDocumentField(XContentType.JSON).v1().getValues()));
final int index = randomFrom(0, 1);
final DocumentField randomCandidate = mutations.get(index).get();
if (!documentField.equals(randomCandidate)) {
if (documentField.equals(randomCandidate) == false) {
return randomCandidate;
} else {
// we are unlucky and our random mutation is equal to our mutation, try the other candidate

View file

@ -79,7 +79,7 @@ public class BoolQueryBuilderTests extends AbstractQueryTestCase<BoolQueryBuilde
@Override
protected void doAssertLuceneQuery(BoolQueryBuilder queryBuilder, Query query, SearchExecutionContext context) throws IOException {
if (!queryBuilder.hasClauses()) {
if (queryBuilder.hasClauses() == false) {
assertThat(query, instanceOf(MatchAllDocsQuery.class));
} else {
List<BooleanClause> clauses = new ArrayList<>();

View file

@ -951,7 +951,7 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
do {
final AllocationId newAllocationId = AllocationId.newInitializing();
// ensure we do not duplicate an allocation ID
if (!existingAllocationIds.contains(newAllocationId)) {
if (existingAllocationIds.contains(newAllocationId) == false) {
return newAllocationId;
}
} while (true);

View file

@ -1568,7 +1568,7 @@ public class IndexShardTests extends IndexShardTestCase {
ElasticsearchException e = expectThrows(ElasticsearchException.class, shard::storeStats);
assertTrue(failureCallbackTriggered.get());
if (corruptIndexException && !throwWhenMarkingStoreCorrupted.get()) {
if (corruptIndexException && throwWhenMarkingStoreCorrupted.get() == false) {
assertTrue(store.isMarkedCorrupted());
}
}
@ -3313,7 +3313,7 @@ public class IndexShardTests extends IndexShardTestCase {
boolean gap = false;
Set<String> ids = new HashSet<>();
for (int i = offset + 1; i < operations; i++) {
if (!rarely() || i == operations - 1) { // last operation can't be a gap as it's not a gap anymore
if (rarely() == false || i == operations - 1) { // last operation can't be a gap as it's not a gap anymore
final String id = ids.isEmpty() || randomBoolean() ? Integer.toString(i) : randomFrom(ids);
if (ids.add(id) == false) { // this is an update
indexShard.advanceMaxSeqNoOfUpdatesOrDeletes(i);
@ -3322,7 +3322,7 @@ public class IndexShardTests extends IndexShardTestCase {
new BytesArray("{}"), XContentType.JSON);
indexShard.applyIndexOperationOnReplica(i, indexShard.getOperationPrimaryTerm(), 1,
IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, sourceToParse);
if (!gap && i == localCheckpoint + 1) {
if (gap == false && i == localCheckpoint + 1) {
localCheckpoint++;
}
max = i;

View file

@ -469,8 +469,8 @@ public class StoreTests extends ESTestCase {
public static void assertConsistent(Store store, Store.MetadataSnapshot metadata) throws IOException {
for (String file : store.directory().listAll()) {
if (!IndexWriter.WRITE_LOCK_NAME.equals(file) &&
!IndexFileNames.OLD_SEGMENTS_GEN.equals(file) && file.startsWith("extra") == false) {
if (IndexWriter.WRITE_LOCK_NAME.equals(file) == false &&
IndexFileNames.OLD_SEGMENTS_GEN.equals(file) == false && file.startsWith("extra") == false) {
assertTrue(file + " is not in the map: " + metadata.asMap().size() + " vs. " +
store.directory().listAll().length, metadata.asMap().containsKey(file));
} else {

View file

@ -3167,7 +3167,7 @@ public class TranslogTests extends ESTestCase {
final long generation = translog.getMinGenerationForSeqNo(seqNo).translogFileGeneration;
int expectedSnapshotOps = 0;
for (long g = generation; g < translog.currentFileGeneration(); g++) {
if (!seqNoPerGeneration.containsKey(g)) {
if (seqNoPerGeneration.containsKey(g) == false) {
final Set<Tuple<Long, Long>> generationSeenSeqNos = new HashSet<>();
int opCount = 0;
final Checkpoint checkpoint = Checkpoint.read(translog.location().resolve(Translog.getCommitCheckpointFileName(g)));

View file

@ -539,7 +539,7 @@ public class HierarchyCircuitBreakerServiceTests extends ESTestCase {
assertThat(output.transientChildUsage, equalTo(input.transientChildUsage));
assertThat(output.permanentChildUsage, equalTo(input.permanentChildUsage));
countDown.get().countDown();
} while (!Thread.interrupted());
} while (Thread.interrupted() == false);
})).collect(Collectors.toList());
threads.forEach(Thread::start);

View file

@ -53,7 +53,9 @@ public class JvmInfoTests extends ESTestCase {
private boolean flagIsEnabled(String argline, String flag) {
final boolean containsPositiveFlag = argline != null && argline.contains("-XX:+" + flag);
if (!containsPositiveFlag) return false;
if (containsPositiveFlag == false) {
return false;
}
final int index = argline.lastIndexOf(flag);
return argline.charAt(index - 1) == '+';
}

View file

@ -123,7 +123,7 @@ public class AggregatorFactoriesTests extends ESTestCase {
word[i] = (char) rand.nextInt(127);
}
name = String.valueOf(word);
if (!matcher.reset(name).matches()) {
if (matcher.reset(name).matches() == false) {
break;
}
}

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