Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import java.util.stream.IntStream;

import static org.apache.lucene.tests.util.TestUtil.randomSimpleString;
import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;
import static org.elasticsearch.core.TimeValue.timeValueSeconds;
import static org.hamcrest.Matchers.instanceOf;

Expand Down Expand Up @@ -174,7 +175,7 @@ private SearchResponse createSearchResponse() {
private void assertSameHits(List<? extends ScrollableHitSource.Hit> actual, SearchHit[] expected) {
assertEquals(actual.size(), expected.length);
for (int i = 0; i < actual.size(); ++i) {
assertEquals(actual.get(i).getSource(), expected[i].getSourceRef());
assertThat(expected[i].getSourceRef(), equalBytes(actual.get(i).getSource()));
assertEquals(actual.get(i).getIndex(), expected[i].getIndex());
assertEquals(actual.get(i).getVersion(), expected[i].getVersion());
assertEquals(actual.get(i).getPrimaryTerm(), expected[i].getPrimaryTerm());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import java.util.Map;

import static org.apache.lucene.tests.util.TestUtil.randomSimpleString;
import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;

/**
* Round trip tests for all {@link Writeable} things declared in this plugin.
Expand Down Expand Up @@ -144,7 +145,7 @@ private void assertRequestEquals(ReindexRequest request, ReindexRequest tripped)
assertNotNull(tripped.getRemoteInfo());
assertEquals(request.getRemoteInfo().getScheme(), tripped.getRemoteInfo().getScheme());
assertEquals(request.getRemoteInfo().getHost(), tripped.getRemoteInfo().getHost());
assertEquals(request.getRemoteInfo().getQuery(), tripped.getRemoteInfo().getQuery());
assertThat(tripped.getRemoteInfo().getQuery(), equalBytes(request.getRemoteInfo().getQuery()));
assertEquals(request.getRemoteInfo().getUsername(), tripped.getRemoteInfo().getUsername());
assertEquals(request.getRemoteInfo().getPassword(), tripped.getRemoteInfo().getPassword());
assertEquals(request.getRemoteInfo().getHeaders(), tripped.getRemoteInfo().getHeaders());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@
import java.util.Collections;
import java.util.Map;

import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;
import static org.elasticsearch.common.io.Streams.readFully;
import static org.elasticsearch.repositories.blobstore.BlobStoreTestUtil.randomPurpose;
import static org.elasticsearch.repositories.gcs.GoogleCloudStorageClientSettings.CREDENTIALS_FILE_SETTING;
Expand Down Expand Up @@ -219,7 +220,7 @@ public void testWriteFileMultipleOfChunkSize() throws IOException {
container.writeBlob(randomPurpose(), key, new BytesArray(initialValue), true);

BytesReference reference = readFully(container.readBlob(randomPurpose(), key));
assertEquals(new BytesArray(initialValue), reference);
assertThat(reference, equalBytes(new BytesArray(initialValue)));

container.deleteBlobsIgnoringIfNotExists(randomPurpose(), Iterators.single(key));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@

import static fixture.gcs.TestUtils.createServiceAccount;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;
import static org.elasticsearch.common.io.Streams.readFully;
import static org.elasticsearch.repositories.blobstore.BlobStoreTestUtil.randomPurpose;
import static org.elasticsearch.repositories.blobstore.ESBlobStoreRepositoryIntegTestCase.randomBytes;
Expand Down Expand Up @@ -582,7 +583,7 @@ public void testCompareAndExchangeWhenThrottled() throws IOException {
final OptionalBytesReference updateResult = safeAwait(
l -> container.compareAndExchangeRegister(randomPurpose(), key, new BytesArray(data), new BytesArray(updatedData), l)
);
assertEquals(new BytesArray(data), updateResult.bytesReference());
assertThat(updateResult.bytesReference(), equalBytes(new BytesArray(data)));

assertEquals(0, requestHandlers.size());
container.delete(randomPurpose());
Expand All @@ -601,7 +602,7 @@ public void testContentsChangeWhileStreaming() throws IOException {
container.writeBlob(randomPurpose(), key, new BytesArray(initialValue), true);

BytesReference reference = readFully(container.readBlob(randomPurpose(), key));
assertEquals(new BytesArray(initialValue), reference);
assertThat(reference, equalBytes(new BytesArray(initialValue)));

try (InputStream inputStream = container.readBlob(randomPurpose(), key)) {
// Trigger the first chunk to load
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import java.util.Map;
import java.util.concurrent.TimeUnit;

import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;
import static org.elasticsearch.repositories.blobstore.BlobStoreTestUtil.randomPurpose;
import static org.elasticsearch.repositories.gcs.GoogleCloudStorageClientSettings.APPLICATION_NAME_SETTING;
import static org.elasticsearch.repositories.gcs.GoogleCloudStorageClientSettings.CONNECT_TIMEOUT_SETTING;
Expand Down Expand Up @@ -138,7 +139,7 @@ public void testSingleMultipartWrite() throws Exception {
final StatsMap wantStats = new StatsMap(purpose);
assertStatsEquals(wantStats.add(INSERT, 1), store.stats());
try (InputStream is = container.readBlob(purpose, blobName)) {
assertEquals(blobContents, Streams.readFully(is));
assertThat(Streams.readFully(is), equalBytes(blobContents));
}
assertStatsEquals(wantStats.add(GET, 1), store.stats());
}
Expand All @@ -164,7 +165,7 @@ public void testResumableWrite() throws Exception {
assertStatsEquals(wantStats.add(INSERT, 1, totalRequests), store.stats());

try (InputStream is = container.readBlob(purpose, blobName)) {
assertEquals(blobContents, Streams.readFully(is));
assertThat(Streams.readFully(is), equalBytes(blobContents));
}
assertStatsEquals(wantStats.add(GET, 1), store.stats());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;

import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;
import static org.elasticsearch.repositories.blobstore.BlobStoreTestUtil.randomPurpose;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.blankOrNullString;
Expand Down Expand Up @@ -194,7 +195,7 @@ List<MultipartUpload> listMultipartUploads() {
assertTrue(testHarness.tryCompareAndSet(BytesArray.EMPTY, bytes1));

// show we're looking at the right blob
assertEquals(bytes1, testHarness.readRegister());
assertThat(testHarness.readRegister(), equalBytes(bytes1));
assertArrayEquals(
bytes1.array(),
client.getObject(GetObjectRequest.builder().bucket(bucketName).key(registerBlobPath).build()).readAllBytes()
Expand All @@ -217,7 +218,7 @@ List<MultipartUpload> listMultipartUploads() {
timeOffsetMillis.addAndGet(blobStore.getCompareAndExchangeTimeToLive().millis() - Math.min(0, age));
assertTrue(testHarness.tryCompareAndSet(bytes1, bytes2));
assertThat(testHarness.listMultipartUploads(), hasSize(0));
assertEquals(bytes2, testHarness.readRegister());
assertThat(testHarness.readRegister(), equalBytes(bytes2));
} finally {
blobContainer.delete(randomPurpose());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.HashMap;
import java.util.Map;

import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;

Expand Down Expand Up @@ -95,6 +96,6 @@ public void testGetResponse() throws Exception {
}
mainResponse.toXContent(responseBuilder, ToXContent.EMPTY_PARAMS);
BytesReference xcontentBytes = BytesReference.bytes(responseBuilder);
assertEquals(xcontentBytes, response.content());
assertThat(response.content(), equalBytes(xcontentBytes));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@

import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_LENGTH;
import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
Expand Down Expand Up @@ -548,7 +549,7 @@ private static void assertChunkedMessageAtIndex(List<Object> messagesSeen, int i
}

private static void assertContentAtIndexEquals(List<Object> messagesSeen, int index, BytesReference single) {
assertEquals(Netty4Utils.toBytesReference(((ByteBufHolder) messagesSeen.get(index)).content()), single);
assertThat(single, equalBytes(Netty4Utils.toBytesReference(((ByteBufHolder) messagesSeen.get(index)).content())));
}

private static void assertDoneWithClosedChannel(ChannelPromise chunkedWritePromise) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;

import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFailures;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
Expand Down Expand Up @@ -95,7 +96,7 @@ public void testBasic() {
refresh("test");
BytesReference pitId = openPointInTime(new String[] { "test" }, TimeValue.timeValueMinutes(2)).getPointInTimeId();
assertResponse(prepareSearch().setPointInTime(new PointInTimeBuilder(pitId)), resp1 -> {
assertThat(resp1.pointInTimeId(), equalTo(pitId));
assertThat(resp1.pointInTimeId(), equalBytes(pitId));
assertHitCount(resp1, numDocs);
});
int deletedDocs = 0;
Expand All @@ -119,7 +120,7 @@ public void testBasic() {
prepareSearch().setQuery(new MatchAllQueryBuilder()).setPointInTime(new PointInTimeBuilder(pitId)),
resp3 -> {
assertHitCount(resp3, numDocs);
assertThat(resp3.pointInTimeId(), equalTo(pitId));
assertThat(resp3.pointInTimeId(), equalBytes(pitId));
}
);
} finally {
Expand Down Expand Up @@ -154,7 +155,7 @@ public void testIndexWithFilteredAlias() {
.setSize(0)
.setQuery(new MatchAllQueryBuilder()),
resp1 -> {
assertThat(resp1.pointInTimeId(), equalTo(pitId));
assertThat(resp1.pointInTimeId(), equalBytes(pitId));
assertHitCount(resp1, finalCountTagA);
}
);
Expand All @@ -181,7 +182,7 @@ public void testMultipleIndices() {
assertNoFailuresAndResponse(prepareSearch().setPointInTime(new PointInTimeBuilder(pitId)), resp -> {
assertHitCount(resp, numDocs);
assertNotNull(resp.pointInTimeId());
assertThat(resp.pointInTimeId(), equalTo(pitId));
assertThat(resp.pointInTimeId(), equalBytes(pitId));
for (int i = 0; i < moreDocs; i++) {
String id = "more-" + i;
String index = "index-" + randomIntBetween(1, numIndices);
Expand All @@ -193,7 +194,7 @@ public void testMultipleIndices() {
assertNoFailuresAndResponse(prepareSearch().setPointInTime(new PointInTimeBuilder(pitId)), resp -> {
assertHitCount(resp, numDocs);
assertNotNull(resp.pointInTimeId());
assertThat(resp.pointInTimeId(), equalTo(pitId));
assertThat(resp.pointInTimeId(), equalBytes(pitId));
});
} finally {
closePointInTime(pitId);
Expand Down Expand Up @@ -237,7 +238,7 @@ public void testIndexFilter() {
assertNoFailuresAndResponse(prepareSearch().setPointInTime(new PointInTimeBuilder(pitId)).setSize(50), resp -> {
assertHitCount(resp, numDocs);
assertNotNull(resp.pointInTimeId());
assertThat(resp.pointInTimeId(), equalTo(pitId));
assertThat(resp.pointInTimeId(), equalBytes(pitId));
for (SearchHit hit : resp.getHits()) {
assertEquals("index-3", hit.getIndex());
}
Expand All @@ -261,7 +262,7 @@ public void testRelocation() throws Exception {
try {
assertNoFailuresAndResponse(prepareSearch().setPointInTime(new PointInTimeBuilder(pitId)), resp -> {
assertHitCount(resp, numDocs);
assertThat(resp.pointInTimeId(), equalTo(pitId));
assertThat(resp.pointInTimeId(), equalBytes(pitId));
});
final Set<String> dataNodes = clusterService().state()
.nodes()
Expand All @@ -281,7 +282,7 @@ public void testRelocation() throws Exception {
}
assertNoFailuresAndResponse(prepareSearch().setPointInTime(new PointInTimeBuilder(pitId)), resp -> {
assertHitCount(resp, numDocs);
assertThat(resp.pointInTimeId(), equalTo(pitId));
assertThat(resp.pointInTimeId(), equalBytes(pitId));
});
assertBusy(() -> {
final Set<String> assignedNodes = clusterService().state()
Expand All @@ -294,7 +295,7 @@ public void testRelocation() throws Exception {
}, 30, TimeUnit.SECONDS);
assertNoFailuresAndResponse(prepareSearch().setPointInTime(new PointInTimeBuilder(pitId)), resp -> {
assertHitCount(resp, numDocs);
assertThat(resp.pointInTimeId(), equalTo(pitId));
assertThat(resp.pointInTimeId(), equalBytes(pitId));
});
} finally {
closePointInTime(pitId);
Expand Down Expand Up @@ -464,13 +465,13 @@ public void testPartialResults() throws Exception {
try {
assertNoFailuresAndResponse(prepareSearch().setPointInTime(new PointInTimeBuilder(pitId)), resp -> {
assertHitCount(resp, numDocs1 + numDocs2);
assertThat(resp.pointInTimeId(), equalTo(pitId));
assertThat(resp.pointInTimeId(), equalBytes(pitId));
});

internalCluster().restartNode(assignedNodeForIndex1);
assertResponse(prepareSearch().setAllowPartialSearchResults(true).setPointInTime(new PointInTimeBuilder(pitId)), resp -> {
assertFailures(resp);
assertThat(resp.pointInTimeId(), equalTo(pitId));
assertThat(resp.pointInTimeId(), equalBytes(pitId));
assertHitCount(resp, numDocs2);
});
} finally {
Expand Down Expand Up @@ -619,7 +620,7 @@ public void testMissingShardsWithPointInTime() throws Exception {
.setPointInTime(new PointInTimeBuilder(pointInTimeResponse.getPointInTimeId())),
resp -> {
// ensure that al docs are returned
assertThat(resp.pointInTimeId(), equalTo(pointInTimeResponse.getPointInTimeId()));
assertThat(resp.pointInTimeId(), equalBytes(pointInTimeResponse.getPointInTimeId()));
assertHitCount(resp, numDocs);
}
);
Expand Down Expand Up @@ -672,7 +673,7 @@ public void testMissingShardsWithPointInTime() throws Exception {
resp -> {
assertThat(resp.getSuccessfulShards(), equalTo(numShards - shardsRemoved));
assertThat(resp.getFailedShards(), equalTo(shardsRemoved));
assertThat(resp.pointInTimeId(), equalTo(pointInTimeResponseOneNodeDown.getPointInTimeId()));
assertThat(resp.pointInTimeId(), equalBytes(pointInTimeResponseOneNodeDown.getPointInTimeId()));
assertNotNull(resp.getHits().getTotalHits());
assertThat(resp.getHits().getTotalHits().value(), lessThan((long) numDocs));
}
Expand Down Expand Up @@ -707,7 +708,7 @@ public void testMissingShardsWithPointInTime() throws Exception {
prepareSearch().setQuery(new MatchAllQueryBuilder())
.setPointInTime(new PointInTimeBuilder(pointInTimeResponseOneNodeDown.getPointInTimeId())),
resp -> {
assertThat(resp.pointInTimeId(), equalTo(pointInTimeResponseOneNodeDown.getPointInTimeId()));
assertThat(resp.pointInTimeId(), equalBytes(pointInTimeResponseOneNodeDown.getPointInTimeId()));
assertThat(resp.getTotalShards(), equalTo(numShards));
assertThat(resp.getSuccessfulShards(), equalTo(numShards - shardsRemoved));
assertThat(resp.getFailedShards(), equalTo(shardsRemoved));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@
import java.util.function.UnaryOperator;

import static java.util.Collections.singleton;
import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder;
Expand Down Expand Up @@ -1026,9 +1027,9 @@ public void testRealTimeGetNestedFields() {
assertTrue(lucene1.isExists());
assertTrue(lucene2.isExists());
assertTrue(lucene3.isExists());
assertThat(translog1.getSourceAsBytesRef(), equalTo(lucene1.getSourceAsBytesRef()));
assertThat(translog2.getSourceAsBytesRef(), equalTo(lucene2.getSourceAsBytesRef()));
assertThat(translog3.getSourceAsBytesRef(), equalTo(lucene3.getSourceAsBytesRef()));
assertThat(translog1.getSourceAsBytesRef(), equalBytes(lucene1.getSourceAsBytesRef()));
assertThat(translog2.getSourceAsBytesRef(), equalBytes(lucene2.getSourceAsBytesRef()));
assertThat(translog3.getSourceAsBytesRef(), equalBytes(lucene3.getSourceAsBytesRef()));
}

private void assertGetFieldsAlwaysWorks(String index, String docId, String[] fields) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@

import static java.util.Collections.singleton;
import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE;
import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;
import static org.elasticsearch.common.util.set.Sets.newHashSet;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
Expand Down Expand Up @@ -664,7 +665,7 @@ public void testStoredFieldsWithoutSource() throws Exception {
assertThat(searchHit.getFields().get("boolean_field").getValue(), equalTo((Object) Boolean.TRUE));
assertThat(
searchHit.getFields().get("binary_field").getValue(),
equalTo(new BytesArray("testing text".getBytes(StandardCharsets.UTF_8)))
equalBytes(new BytesArray("testing text".getBytes(StandardCharsets.UTF_8)))
);
}
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@
import java.util.HashMap;
import java.util.Map;

import static org.elasticsearch.common.bytes.BytesReferenceTestUtils.equalBytes;

public class GetFieldMappingsResponseTests extends AbstractWireSerializingTestCase<GetFieldMappingsResponse> {

public void testManualSerialization() throws IOException {
Expand All @@ -36,7 +38,7 @@ public void testManualSerialization() throws IOException {
GetFieldMappingsResponse serialized = new GetFieldMappingsResponse(in);
FieldMappingMetadata metadata = serialized.fieldMappings("index", "field");
assertNotNull(metadata);
assertEquals(new BytesArray("{}"), metadata.source());
assertThat(metadata.source(), equalBytes(new BytesArray("{}")));
}
}
}
Expand Down
Loading