mirror of https://github.com/apache/lucene.git
LUCENE-9145 First pass addressing static analysis (#1181)
Fixed a bunch of the smaller warnings found by error-prone compiler plugin, while ignoring a lot of the bigger ones.
This commit is contained in:
parent
8147e491ce
commit
338d386ae0
|
@ -96,7 +96,7 @@ public abstract class TrecDocParser {
|
|||
*/
|
||||
public static String stripTags(String buf, int start) {
|
||||
if (start>0) {
|
||||
buf = buf.substring(0);
|
||||
buf = buf.substring(start);
|
||||
}
|
||||
return buf.replaceAll("<[^>]*>", " ");
|
||||
}
|
||||
|
|
|
@ -66,11 +66,10 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.NamedThreadFactory;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.TimeUnits;
|
||||
import org.junit.Test;
|
||||
|
||||
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "none")
|
||||
@TimeoutSuite(millis = 365 * 24 * TimeUnits.HOUR) // hopefully ~1 year is long enough ;)
|
||||
@TimeoutSuite(millis = Integer.MAX_VALUE) // hopefully ~24 days is long enough ;)
|
||||
@LuceneTestCase.Monster("takes a lot!")
|
||||
public final class Test20NewsgroupsClassification extends LuceneTestCase {
|
||||
|
||||
|
@ -337,4 +336,4 @@ public final class Test20NewsgroupsClassification extends LuceneTestCase {
|
|||
return number;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
package org.apache.lucene.search;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.atomic.LongAccumulator;
|
||||
|
||||
/**
|
||||
|
@ -79,6 +80,11 @@ final class MaxScoreAccumulator {
|
|||
Float.compare(result.score, score) == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(docID, score);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DocAndScore{" +
|
||||
|
|
|
@ -235,7 +235,7 @@ public abstract class StringHelper {
|
|||
// may not be available on this platform
|
||||
// fall back to lower quality randomness from 3 different sources:
|
||||
x0 = System.nanoTime();
|
||||
x1 = StringHelper.class.hashCode() << 32;
|
||||
x1 = (long) StringHelper.class.hashCode() << 32;
|
||||
|
||||
StringBuilder sb = new StringBuilder();
|
||||
// Properties can vary across JVM instances:
|
||||
|
|
|
@ -211,9 +211,8 @@ public class TestCharArraySet extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecated")
|
||||
public void testCopyCharArraySetBWCompat() {
|
||||
CharArraySet setIngoreCase = new CharArraySet(10, true);
|
||||
CharArraySet setIgnoreCase = new CharArraySet(10, true);
|
||||
CharArraySet setCaseSensitive = new CharArraySet(10, false);
|
||||
|
||||
List<String> stopwords = Arrays.asList(TEST_STOP_WORDS);
|
||||
|
@ -221,15 +220,15 @@ public class TestCharArraySet extends LuceneTestCase {
|
|||
for (String string : stopwords) {
|
||||
stopwordsUpper.add(string.toUpperCase(Locale.ROOT));
|
||||
}
|
||||
setIngoreCase.addAll(Arrays.asList(TEST_STOP_WORDS));
|
||||
setIngoreCase.add(Integer.valueOf(1));
|
||||
setIgnoreCase.addAll(Arrays.asList(TEST_STOP_WORDS));
|
||||
setIgnoreCase.add(Integer.valueOf(1));
|
||||
setCaseSensitive.addAll(Arrays.asList(TEST_STOP_WORDS));
|
||||
setCaseSensitive.add(Integer.valueOf(1));
|
||||
|
||||
CharArraySet copy = CharArraySet.copy(setIngoreCase);
|
||||
CharArraySet copy = CharArraySet.copy(setIgnoreCase);
|
||||
CharArraySet copyCaseSens = CharArraySet.copy(setCaseSensitive);
|
||||
|
||||
assertEquals(setIngoreCase.size(), copy.size());
|
||||
assertEquals(setIgnoreCase.size(), copy.size());
|
||||
assertEquals(setCaseSensitive.size(), copy.size());
|
||||
|
||||
assertTrue(copy.containsAll(stopwords));
|
||||
|
@ -250,7 +249,7 @@ public class TestCharArraySet extends LuceneTestCase {
|
|||
assertTrue(copy.containsAll(newWords));
|
||||
// new added terms are not in the source set
|
||||
for (String string : newWords) {
|
||||
assertFalse(setIngoreCase.contains(string));
|
||||
assertFalse(setIgnoreCase.contains(string));
|
||||
assertFalse(setCaseSensitive.contains(string));
|
||||
|
||||
}
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.lucene.util.LuceneTestCase.Monster;
|
|||
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.TimeUnits;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
||||
|
||||
|
@ -36,7 +35,7 @@ import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
|||
// or: python -u /l/util/src/python/repeatLuceneTest.py -heap 6g -once -nolog -tmpDir /b/tmp -logDir /l/logs Test2BPoints.test2D -verbose
|
||||
|
||||
@SuppressCodecs({ "SimpleText", "Direct", "Compressing" })
|
||||
@TimeoutSuite(millis = 365 * 24 * TimeUnits.HOUR) // hopefully ~1 year is long enough ;)
|
||||
@TimeoutSuite(millis = Integer.MAX_VALUE) // hopefully ~24 days is long enough ;)
|
||||
@Monster("takes at least 4 hours and consumes many GB of temp disk space")
|
||||
public class Test2BPoints extends LuceneTestCase {
|
||||
public void test1D() throws Exception {
|
||||
|
|
|
@ -48,8 +48,8 @@ public class TestFieldReuse extends BaseTokenStreamTestCase {
|
|||
// now reuse previous stream
|
||||
stringField = new StringField("foo", "baz", Field.Store.NO);
|
||||
TokenStream ts2 = stringField.tokenStream(null, ts);
|
||||
assertSame(ts, ts);
|
||||
assertTokenStreamContents(ts,
|
||||
assertSame(ts, ts2);
|
||||
assertTokenStreamContents(ts,
|
||||
new String[] { "baz" },
|
||||
new int[] { 0 },
|
||||
new int[] { 3 }
|
||||
|
|
|
@ -38,7 +38,7 @@ public class TestTransactions extends LuceneTestCase {
|
|||
private class RandomFailure extends MockDirectoryWrapper.Failure {
|
||||
@Override
|
||||
public void eval(MockDirectoryWrapper dir) throws IOException {
|
||||
if (TestTransactions.doFail && random().nextInt() % 10 <= 3) {
|
||||
if (TestTransactions.doFail && random().nextInt(10) <= 3) {
|
||||
if (VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + " TEST: now fail on purpose");
|
||||
new Throwable().printStackTrace(System.out);
|
||||
|
|
|
@ -272,7 +272,7 @@ public class TestBufferedIndexInput extends LuceneTestCase {
|
|||
//int count = 0;
|
||||
for (final IndexInput ip : allIndexInputs) {
|
||||
BufferedIndexInput bii = (BufferedIndexInput) ip;
|
||||
int bufferSize = 1024+Math.abs(rand.nextInt() % 32768);
|
||||
int bufferSize = 1024 + rand.nextInt(32768);
|
||||
bii.setBufferSize(bufferSize);
|
||||
//count++;
|
||||
}
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.lucene.store.IndexOutput;
|
|||
import org.apache.lucene.util.LuceneTestCase.Monster;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.NumericUtils;
|
||||
import org.apache.lucene.util.TimeUnits;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
||||
|
||||
|
@ -33,7 +32,7 @@ import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
|||
//
|
||||
// or: python -u /l/util/src/python/repeatLuceneTest.py -heap 4g -once -nolog -tmpDir /b/tmp -logDir /l/logs Test2BBKDPoints.test2D -verbose
|
||||
|
||||
@TimeoutSuite(millis = 365 * 24 * TimeUnits.HOUR) // hopefully ~1 year is long enough ;)
|
||||
@TimeoutSuite(millis = Integer.MAX_VALUE) // hopefully ~24 days is long enough ;)
|
||||
@Monster("takes at least 4 hours and consumes many GB of temp disk space")
|
||||
public class Test2BBKDPoints extends LuceneTestCase {
|
||||
public void test1D() throws Exception {
|
||||
|
|
|
@ -170,7 +170,7 @@ public class ConcurrentQueryLoader implements Closeable {
|
|||
*/
|
||||
private static <E> int drain(BlockingQueue<E> q, Collection<? super E> buffer, int numElements,
|
||||
long timeout, TimeUnit unit) throws InterruptedException {
|
||||
buffer = Objects.requireNonNull(buffer);
|
||||
Objects.requireNonNull(buffer);
|
||||
/*
|
||||
* This code performs one System.nanoTime() more than necessary, and in return, the time to
|
||||
* execute Queue#drainTo is not added *on top* of waiting for the timeout (which could make
|
||||
|
|
|
@ -89,7 +89,7 @@ public class TestLatLonShape extends LuceneTestCase {
|
|||
|
||||
Polygon polygon;
|
||||
Document document;
|
||||
for (int i = 0; i < numPolys;) {
|
||||
for (int i = 0; i < numPolys; i++) {
|
||||
document = new Document();
|
||||
numVertices = TestUtil.nextInt(random(), 100000, 200000);
|
||||
polygon = GeoTestUtil.createRegularPolygon(0, 0, atLeast(1000000), numVertices);
|
||||
|
|
|
@ -72,13 +72,13 @@ public abstract class MockFileSystemTestCase extends LuceneTestCase {
|
|||
assumeFalse("broken on J9: see https://issues.apache.org/jira/browse/LUCENE-6517", Constants.JAVA_VENDOR.startsWith("IBM"));
|
||||
Path dir = wrap(createTempDir());
|
||||
|
||||
Path f1 = null;
|
||||
try {
|
||||
dir.resolve(fileName);
|
||||
f1 = dir.resolve(fileName);
|
||||
} catch (InvalidPathException ipe) {
|
||||
assumeNoException("couldn't resolve '"+fileName+"'", ipe);
|
||||
}
|
||||
|
||||
Path f1 = dir.resolve(fileName);
|
||||
URI uri = f1.toUri();
|
||||
Path f2 = dir.getFileSystem().provider().getPath(uri);
|
||||
assertEquals(f1, f2);
|
||||
|
|
|
@ -185,7 +185,8 @@ public class ZkContainer {
|
|||
try {
|
||||
try {
|
||||
if (testing_beforeRegisterInZk != null) {
|
||||
testing_beforeRegisterInZk.test(cd);
|
||||
boolean didTrigger = testing_beforeRegisterInZk.test(cd);
|
||||
log.debug((didTrigger ? "Ran" : "Skipped") + " pre-zk hook");
|
||||
}
|
||||
if (!core.getCoreContainer().isShutDown()) {
|
||||
zkController.register(core.getName(), cd, skipRecovery);
|
||||
|
|
|
@ -60,6 +60,7 @@ import java.util.stream.Stream;
|
|||
import java.util.zip.ZipEntry;
|
||||
import java.util.zip.ZipInputStream;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
import org.apache.commons.cli.GnuParser;
|
||||
import org.apache.commons.cli.HelpFormatter;
|
||||
|
@ -1424,19 +1425,17 @@ public class SolrCLI implements CLIO {
|
|||
private static final long MS_IN_HOUR = MS_IN_MIN * 60L;
|
||||
private static final long MS_IN_DAY = MS_IN_HOUR * 24L;
|
||||
|
||||
private static final String uptime(long uptimeMs) {
|
||||
@VisibleForTesting
|
||||
static final String uptime(long uptimeMs) {
|
||||
if (uptimeMs <= 0L) return "?";
|
||||
|
||||
long numDays = (uptimeMs >= MS_IN_DAY)
|
||||
? (long) Math.floor(uptimeMs / MS_IN_DAY) : 0L;
|
||||
long numDays = (uptimeMs >= MS_IN_DAY) ? (uptimeMs / MS_IN_DAY) : 0L;
|
||||
long rem = uptimeMs - (numDays * MS_IN_DAY);
|
||||
long numHours = (rem >= MS_IN_HOUR)
|
||||
? (long) Math.floor(rem / MS_IN_HOUR) : 0L;
|
||||
long numHours = (rem >= MS_IN_HOUR) ? (rem / MS_IN_HOUR) : 0L;
|
||||
rem = rem - (numHours * MS_IN_HOUR);
|
||||
long numMinutes = (rem >= MS_IN_MIN)
|
||||
? (long) Math.floor(rem / MS_IN_MIN) : 0L;
|
||||
long numMinutes = (rem >= MS_IN_MIN) ? (rem / MS_IN_MIN) : 0L;
|
||||
rem = rem - (numMinutes * MS_IN_MIN);
|
||||
long numSeconds = Math.round(rem / 1000);
|
||||
long numSeconds = Math.round(rem / 1000.0);
|
||||
return String.format(Locale.ROOT, "%d days, %d hours, %d minutes, %d seconds", numDays,
|
||||
numHours, numMinutes, numSeconds);
|
||||
}
|
||||
|
|
|
@ -187,7 +187,6 @@ public class CdcrBidirectionalTest extends SolrTestCaseJ4 {
|
|||
req = new UpdateRequest();
|
||||
doc = new SolrInputDocument();
|
||||
String atomicFieldName = "abc";
|
||||
ImmutableMap.of("", "");
|
||||
String atomicUpdateId = "cluster2_" + random().nextInt(numDocs_c2);
|
||||
doc.addField("id", atomicUpdateId);
|
||||
doc.addField("xyz", ImmutableMap.of("delete", ""));
|
||||
|
|
|
@ -128,7 +128,7 @@ public class CdcrTestsUtil extends SolrTestCaseJ4 {
|
|||
}
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
return response != null ? response.getResults().getNumFound() : null;
|
||||
return response != null ? response.getResults().getNumFound() : 0;
|
||||
}
|
||||
|
||||
protected static boolean assertShardInSync(String collection, String shard, CloudSolrClient client) throws IOException, SolrServerException {
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.util.Arrays;
|
|||
import java.util.List;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.function.UnaryOperator;
|
||||
|
||||
import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
|
||||
import org.apache.solr.client.solrj.SolrClient;
|
||||
|
@ -32,7 +31,6 @@ import org.apache.solr.common.SolrException;
|
|||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.SolrInputDocument;
|
||||
import org.apache.solr.util.BaseTestHarness;
|
||||
import org.apache.solr.util.RestTestHarness;
|
||||
import org.eclipse.jetty.servlet.ServletHolder;
|
||||
import org.junit.After;
|
||||
import org.junit.Test;
|
||||
|
@ -118,22 +116,18 @@ public class TestCloudSchemaless extends AbstractFullDistribZkTestBase {
|
|||
|
||||
String [] expectedFields = getExpectedFieldResponses(docNumber);
|
||||
// Check that all the fields were added
|
||||
forAllRestTestHarnesses( new UnaryOperator<RestTestHarness>() {
|
||||
@Override
|
||||
public RestTestHarness apply(RestTestHarness client) {
|
||||
try {
|
||||
String request = "/schema/fields?wt=xml";
|
||||
String response = client.query(request);
|
||||
String result = BaseTestHarness.validateXPath(response, expectedFields);
|
||||
if (result != null) {
|
||||
String msg = "QUERY FAILED: xpath=" + result + " request=" + request + " response=" + response;
|
||||
log.error(msg);
|
||||
fail(msg);
|
||||
}
|
||||
} catch (Exception ex) {
|
||||
fail("Caught exception: "+ex);
|
||||
forAllRestTestHarnesses(client -> {
|
||||
try {
|
||||
String request = "/schema/fields?wt=xml";
|
||||
String response = client.query(request);
|
||||
String result = BaseTestHarness.validateXPath(response, expectedFields);
|
||||
if (result != null) {
|
||||
String msg = "QUERY FAILED: xpath=" + result + " request=" + request + " response=" + response;
|
||||
log.error(msg);
|
||||
fail(msg);
|
||||
}
|
||||
return client;
|
||||
} catch (Exception ex) {
|
||||
fail("Caught exception: "+ex);
|
||||
}
|
||||
});
|
||||
|
||||
|
|
|
@ -2216,7 +2216,7 @@ public class TestPointFields extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
private List<Integer> getRandomInts(int length, boolean missingVals, int bound) {
|
||||
return getRandomList(length, missingVals, () -> random().nextInt() % bound);
|
||||
return getRandomList(length, missingVals, () -> random().nextInt(bound));
|
||||
}
|
||||
|
||||
private List<Integer> getRandomInts(int length, boolean missingVals) {
|
||||
|
|
|
@ -20,7 +20,7 @@ import java.io.IOException;
|
|||
import java.lang.invoke.MethodHandles;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -31,14 +31,14 @@ public class MockAuthorizationPlugin implements AuthorizationPlugin {
|
|||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||
static final HashSet<String> denyUsers = new HashSet<>();
|
||||
static final HashSet<String> protectedResources = new HashSet<>();
|
||||
static Predicate<AuthorizationContext> predicate;
|
||||
static Consumer<AuthorizationContext> predicate;
|
||||
|
||||
@Override
|
||||
public AuthorizationResponse authorize(AuthorizationContext context) {
|
||||
String uname = context.getUserPrincipal() == null ? null : context.getUserPrincipal().getName();
|
||||
if (predicate != null) {
|
||||
try {
|
||||
predicate.test(context);
|
||||
predicate.accept(context);
|
||||
return new AuthorizationResponse(200);
|
||||
} catch (SolrException e) {
|
||||
return new AuthorizationResponse(e.code());
|
||||
|
|
|
@ -83,7 +83,6 @@ public class PKIAuthenticationIntegrationTest extends SolrCloudAuthTestCase {
|
|||
count.incrementAndGet();
|
||||
}
|
||||
}
|
||||
return true;
|
||||
};
|
||||
|
||||
MockAuthenticationPlugin.predicate = servletRequest -> {
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.function.UnaryOperator;
|
||||
import java.util.zip.ZipEntry;
|
||||
import java.util.zip.ZipOutputStream;
|
||||
|
||||
|
@ -80,20 +79,16 @@ public class TestNamedUpdateProcessors extends AbstractFullDistribZkTestBase {
|
|||
|
||||
client = randomRestTestHarness();
|
||||
TestSolrConfigHandler.runConfigCommand(client, "/config", payload);
|
||||
forAllRestTestHarnesses( new UnaryOperator<RestTestHarness>() {
|
||||
@Override
|
||||
public RestTestHarness apply(RestTestHarness restTestHarness) {
|
||||
try {
|
||||
TestSolrConfigHandler.testForResponseElement(restTestHarness,
|
||||
null,
|
||||
"/config/overlay",
|
||||
null,
|
||||
Arrays.asList("overlay", "updateProcessor", "firstFld", "fieldName"),
|
||||
"test_s", 10);
|
||||
} catch (Exception ex) {
|
||||
fail("Caught exception: "+ex);
|
||||
}
|
||||
return restTestHarness;
|
||||
forAllRestTestHarnesses(restTestHarness -> {
|
||||
try {
|
||||
TestSolrConfigHandler.testForResponseElement(restTestHarness,
|
||||
null,
|
||||
"/config/overlay",
|
||||
null,
|
||||
Arrays.asList("overlay", "updateProcessor", "firstFld", "fieldName"),
|
||||
"test_s", 10);
|
||||
} catch (Exception ex) {
|
||||
fail("Caught exception: "+ex);
|
||||
}
|
||||
});
|
||||
|
||||
|
|
|
@ -0,0 +1,36 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.solr.util;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class SolrCliUptimeTest {
|
||||
@Test
|
||||
public void testUptime() {
|
||||
assertEquals("?", SolrCLI.uptime(0));
|
||||
assertEquals("0 days, 0 hours, 0 minutes, 0 seconds", SolrCLI.uptime(1));
|
||||
|
||||
assertEquals("Should have rounded down", "0 days, 0 hours, 0 minutes, 0 seconds", SolrCLI.uptime(499));
|
||||
assertEquals("Should have rounded up", "0 days, 0 hours, 0 minutes, 1 seconds", SolrCLI.uptime(501));
|
||||
|
||||
// Overflow
|
||||
assertEquals("24 days, 20 hours, 31 minutes, 24 seconds", SolrCLI.uptime(Integer.MAX_VALUE));
|
||||
assertEquals("106751991167 days, 7 hours, 12 minutes, 56 seconds", SolrCLI.uptime(Long.MAX_VALUE));
|
||||
}
|
||||
}
|
|
@ -44,8 +44,7 @@ public enum FieldFlag {
|
|||
|
||||
FieldFlag(char abbreviation, String display) {
|
||||
this.abbreviation = abbreviation;
|
||||
this.display = display;
|
||||
this.display.intern();//QUESTION: Need we bother here?
|
||||
this.display = display.intern();//QUESTION: Need we bother here?
|
||||
}
|
||||
|
||||
public static FieldFlag getFlag(char abbrev){
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.io.OutputStream;
|
|||
/**
|
||||
* A byte[] backed String
|
||||
*/
|
||||
public interface Utf8CharSequence extends CharSequence , Comparable, Cloneable {
|
||||
public interface Utf8CharSequence extends CharSequence , Comparable<Utf8CharSequence>, Cloneable {
|
||||
|
||||
/**
|
||||
* Write the bytes into a buffer. The objective is to avoid the local bytes being exposed to
|
||||
|
@ -48,7 +48,7 @@ public interface Utf8CharSequence extends CharSequence , Comparable, Cloneable {
|
|||
byte byteAt(int idx);
|
||||
|
||||
@Override
|
||||
default int compareTo(Object o) {
|
||||
default int compareTo(Utf8CharSequence o) {
|
||||
if(o == null) return 1;
|
||||
return toString().compareTo(o.toString());
|
||||
}
|
||||
|
|
|
@ -41,7 +41,7 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.UnaryOperator;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.solr.client.solrj.SolrClient;
|
||||
|
@ -1822,7 +1822,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
|||
cloudClient.waitForState(collectionName, 30, TimeUnit.SECONDS, SolrCloudTestCase.activeClusterShape(numShards,
|
||||
numShards * (numNrtReplicas + numTlogReplicas + numPullReplicas)));
|
||||
} catch (TimeoutException e) {
|
||||
new RuntimeException("Timeout waiting for " + numShards + " shards and " + (numNrtReplicas + numTlogReplicas + numPullReplicas) + " replicas.", e);
|
||||
throw new RuntimeException("Timeout waiting for " + numShards + " shards and " + (numNrtReplicas + numTlogReplicas + numPullReplicas) + " replicas.", e);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
@ -2422,10 +2422,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
|||
return restTestHarnesses.get(random.nextInt(restTestHarnesses.size()));
|
||||
}
|
||||
|
||||
protected void forAllRestTestHarnesses(UnaryOperator<RestTestHarness> op) {
|
||||
for (RestTestHarness h : restTestHarnesses) {
|
||||
op.apply(h);
|
||||
}
|
||||
protected void forAllRestTestHarnesses(Consumer<RestTestHarness> op) {
|
||||
restTestHarnesses.forEach(op);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue