Java API: Make sure afterBulk is always called in BulkProcessor after beforeBulk

Moved BulkProcessor tests from BulkTests to newly added BulkProcessorTests class.
Strenghtened BulkProcessorTests by adding randomizations to existing tests and new tests for concurrent requests and expcetions.
Also made sure that afterBulk is called only once per request if concurrentRequests==0.

Closes #5038
This commit is contained in:
javanna 2014-06-12 14:20:18 +02:00 committed by Luca Cavanna
parent 44097b358d
commit b9ffb2b0a5
3 changed files with 336 additions and 99 deletions

View File

@ -276,17 +276,22 @@ public class BulkProcessor implements Closeable {
if (concurrentRequests == 0) {
// execute in a blocking fashion...
boolean afterCalled = false;
try {
listener.beforeBulk(executionId, bulkRequest);
listener.afterBulk(executionId, bulkRequest, client.bulk(bulkRequest).actionGet());
BulkResponse bulkItemResponses = client.bulk(bulkRequest).actionGet();
afterCalled = true;
listener.afterBulk(executionId, bulkRequest, bulkItemResponses);
} catch (Exception e) {
listener.afterBulk(executionId, bulkRequest, e);
if (!afterCalled) {
listener.afterBulk(executionId, bulkRequest, e);
}
}
} else {
boolean success = false;
try {
semaphore.acquire();
listener.beforeBulk(executionId, bulkRequest);
semaphore.acquire();
client.bulk(bulkRequest, new ActionListener<BulkResponse>() {
@Override
public void onResponse(BulkResponse response) {
@ -310,12 +315,13 @@ public class BulkProcessor implements Closeable {
} catch (InterruptedException e) {
Thread.interrupted();
listener.afterBulk(executionId, bulkRequest, e);
} catch (Throwable t) {
listener.afterBulk(executionId, bulkRequest, t);
} finally {
if (!success) { // if we fail on client.bulk() release the semaphore
semaphore.release();
}
}
}
}

View File

@ -0,0 +1,326 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.action.bulk;
import org.elasticsearch.action.get.MultiGetItemResponse;
import org.elasticsearch.action.get.MultiGetRequestBuilder;
import org.elasticsearch.action.get.MultiGetResponse;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.transport.TransportClient;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.*;
public class BulkProcessorTests extends ElasticsearchIntegrationTest {
@Test
public void testThatBulkProcessorCountIsCorrect() throws InterruptedException {
final CountDownLatch latch = new CountDownLatch(1);
BulkProcessorTestListener listener = new BulkProcessorTestListener(latch);
int numDocs = randomIntBetween(10, 100);
try (BulkProcessor processor = BulkProcessor.builder(client(), listener).setName("foo")
//let's make sure that the bulk action limit trips, one single execution will index all the documents
.setConcurrentRequests(randomIntBetween(0, 1)).setBulkActions(numDocs)
.setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(1, ByteSizeUnit.GB))
.build()) {
MultiGetRequestBuilder multiGetRequestBuilder = indexDocs(client(), processor, numDocs);
latch.await();
assertThat(listener.beforeCounts.get(), equalTo(1));
assertThat(listener.afterCounts.get(), equalTo(1));
assertThat(listener.bulkFailures.size(), equalTo(0));
assertResponseItems(listener.bulkItems, numDocs);
assertMultiGetResponse(multiGetRequestBuilder.get(), numDocs);
}
}
@Test
public void testBulkProcessorFlush() throws InterruptedException {
final CountDownLatch latch = new CountDownLatch(1);
BulkProcessorTestListener listener = new BulkProcessorTestListener(latch);
int numDocs = randomIntBetween(10, 100);
try (BulkProcessor processor = BulkProcessor.builder(client(), listener).setName("foo")
//let's make sure that this bulk won't be automatically flushed
.setConcurrentRequests(randomIntBetween(0, 10)).setBulkActions(numDocs + randomIntBetween(1, 100))
.setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(1, ByteSizeUnit.GB)).build()) {
MultiGetRequestBuilder multiGetRequestBuilder = indexDocs(client(), processor, numDocs);
assertThat(latch.await(randomInt(500), TimeUnit.MILLISECONDS), equalTo(false));
//we really need an explicit flush as none of the bulk thresholds was reached
processor.flush();
latch.await();
assertThat(listener.beforeCounts.get(), equalTo(1));
assertThat(listener.afterCounts.get(), equalTo(1));
assertThat(listener.bulkFailures.size(), equalTo(0));
assertResponseItems(listener.bulkItems, numDocs);
assertMultiGetResponse(multiGetRequestBuilder.get(), numDocs);
}
}
@Test
public void testBulkProcessorConcurrentRequests() throws Exception {
int bulkActions = randomIntBetween(10, 100);
int numDocs = randomIntBetween(bulkActions, bulkActions + 100);
int concurrentRequests = randomIntBetween(0, 10);
int expectedBulkActions = numDocs / bulkActions;
final CountDownLatch latch = new CountDownLatch(expectedBulkActions);
int totalExpectedBulkActions = numDocs % bulkActions == 0 ? expectedBulkActions : expectedBulkActions + 1;
final CountDownLatch closeLatch = new CountDownLatch(totalExpectedBulkActions);
BulkProcessorTestListener listener = new BulkProcessorTestListener(latch, closeLatch);
MultiGetRequestBuilder multiGetRequestBuilder;
try (BulkProcessor processor = BulkProcessor.builder(client(), listener)
.setConcurrentRequests(concurrentRequests).setBulkActions(bulkActions)
//set interval and size to high values
.setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(1, ByteSizeUnit.GB)).build()) {
multiGetRequestBuilder = indexDocs(client(), processor, numDocs);
latch.await();
assertThat(listener.beforeCounts.get(), equalTo(expectedBulkActions));
assertThat(listener.afterCounts.get(), equalTo(expectedBulkActions));
assertThat(listener.bulkFailures.size(), equalTo(0));
assertThat(listener.bulkItems.size(), equalTo(numDocs - numDocs % bulkActions));
}
closeLatch.await();
assertThat(listener.beforeCounts.get(), equalTo(totalExpectedBulkActions));
assertThat(listener.afterCounts.get(), equalTo(totalExpectedBulkActions));
assertThat(listener.bulkFailures.size(), equalTo(0));
assertThat(listener.bulkItems.size(), equalTo(numDocs));
Set<String> ids = new HashSet<>();
for (BulkItemResponse bulkItemResponse : listener.bulkItems) {
assertThat(bulkItemResponse.isFailed(), equalTo(false));
assertThat(bulkItemResponse.getIndex(), equalTo("test"));
assertThat(bulkItemResponse.getType(), equalTo("test"));
//with concurrent requests > 1 we can't rely on the order of the bulk requests
assertThat(Integer.valueOf(bulkItemResponse.getId()), both(greaterThan(0)).and(lessThanOrEqualTo(numDocs)));
//we do want to check that we don't get duplicate ids back
assertThat(ids.add(bulkItemResponse.getId()), equalTo(true));
}
assertMultiGetResponse(multiGetRequestBuilder.get(), numDocs);
}
@Test
//https://github.com/elasticsearch/elasticsearch/issues/5038
public void testBulkProcessorConcurrentRequestsNoNodeAvailableException() throws Exception {
//we create a transport client with no nodes to make sure it throws NoNodeAvailableException
Client transportClient = new TransportClient();
int bulkActions = randomIntBetween(10, 100);
int numDocs = randomIntBetween(bulkActions, bulkActions + 100);
int concurrentRequests = randomIntBetween(0, 10);
int expectedBulkActions = numDocs / bulkActions;
final CountDownLatch latch = new CountDownLatch(expectedBulkActions);
int totalExpectedBulkActions = numDocs % bulkActions == 0 ? expectedBulkActions : expectedBulkActions + 1;
final CountDownLatch closeLatch = new CountDownLatch(totalExpectedBulkActions);
BulkProcessorTestListener listener = new BulkProcessorTestListener(latch, closeLatch);
try (BulkProcessor processor = BulkProcessor.builder(transportClient, listener)
.setConcurrentRequests(concurrentRequests).setBulkActions(bulkActions)
//set interval and size to high values
.setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(1, ByteSizeUnit.GB)).build()) {
indexDocs(transportClient, processor, numDocs);
latch.await();
assertThat(listener.beforeCounts.get(), equalTo(expectedBulkActions));
assertThat(listener.afterCounts.get(), equalTo(expectedBulkActions));
assertThat(listener.bulkFailures.size(), equalTo(expectedBulkActions));
assertThat(listener.bulkItems.size(), equalTo(0));
}
closeLatch.await();
assertThat(listener.bulkFailures.size(), equalTo(totalExpectedBulkActions));
assertThat(listener.bulkItems.size(), equalTo(0));
}
@Test
public void testBulkProcessorConcurrentRequestsReadOnlyIndex() throws Exception {
createIndex("test-ro");
assertAcked(client().admin().indices().prepareUpdateSettings("test-ro")
.setSettings(ImmutableSettings.builder().put("index.blocks.read_only", true)));
ensureGreen();
int bulkActions = randomIntBetween(10, 100);
int numDocs = randomIntBetween(bulkActions, bulkActions + 100);
int concurrentRequests = randomIntBetween(0, 10);
int expectedBulkActions = numDocs / bulkActions;
final CountDownLatch latch = new CountDownLatch(expectedBulkActions);
int totalExpectedBulkActions = numDocs % bulkActions == 0 ? expectedBulkActions : expectedBulkActions + 1;
final CountDownLatch closeLatch = new CountDownLatch(totalExpectedBulkActions);
int testDocs = 0;
int testReadOnlyDocs = 0;
MultiGetRequestBuilder multiGetRequestBuilder = client().prepareMultiGet();
BulkProcessorTestListener listener = new BulkProcessorTestListener(latch, closeLatch);
try (BulkProcessor processor = BulkProcessor.builder(client(), listener)
.setConcurrentRequests(concurrentRequests).setBulkActions(bulkActions)
//set interval and size to high values
.setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(1, ByteSizeUnit.GB)).build()) {
for (int i = 1; i <= numDocs; i++) {
if (randomBoolean()) {
testDocs++;
processor.add(new IndexRequest("test", "test", Integer.toString(testDocs)).source("field", "value"));
multiGetRequestBuilder.add("test", "test", Integer.toString(testDocs));
} else {
testReadOnlyDocs++;
processor.add(new IndexRequest("test-ro", "test", Integer.toString(testReadOnlyDocs)).source("field", "value"));
}
}
}
closeLatch.await();
assertThat(listener.beforeCounts.get(), equalTo(totalExpectedBulkActions));
assertThat(listener.afterCounts.get(), equalTo(totalExpectedBulkActions));
assertThat(listener.bulkFailures.size(), equalTo(0));
assertThat(listener.bulkItems.size(), equalTo(testDocs + testReadOnlyDocs));
Set<String> ids = new HashSet<>();
Set<String> readOnlyIds = new HashSet<>();
for (BulkItemResponse bulkItemResponse : listener.bulkItems) {
assertThat(bulkItemResponse.getIndex(), either(equalTo("test")).or(equalTo("test-ro")));
assertThat(bulkItemResponse.getType(), equalTo("test"));
if (bulkItemResponse.getIndex().equals("test")) {
assertThat(bulkItemResponse.isFailed(), equalTo(false));
//with concurrent requests > 1 we can't rely on the order of the bulk requests
assertThat(Integer.valueOf(bulkItemResponse.getId()), both(greaterThan(0)).and(lessThanOrEqualTo(testDocs)));
//we do want to check that we don't get duplicate ids back
assertThat(ids.add(bulkItemResponse.getId()), equalTo(true));
} else {
assertThat(bulkItemResponse.isFailed(), equalTo(true));
//with concurrent requests > 1 we can't rely on the order of the bulk requests
assertThat(Integer.valueOf(bulkItemResponse.getId()), both(greaterThan(0)).and(lessThanOrEqualTo(testReadOnlyDocs)));
//we do want to check that we don't get duplicate ids back
assertThat(readOnlyIds.add(bulkItemResponse.getId()), equalTo(true));
}
}
assertMultiGetResponse(multiGetRequestBuilder.get(), testDocs);
}
private static MultiGetRequestBuilder indexDocs(Client client, BulkProcessor processor, int numDocs) {
MultiGetRequestBuilder multiGetRequestBuilder = client.prepareMultiGet();
for (int i = 1; i <= numDocs; i++) {
processor.add(new IndexRequest("test", "test", Integer.toString(i)).source("field", randomRealisticUnicodeOfLengthBetween(1, 30)));
multiGetRequestBuilder.add("test", "test", Integer.toString(i));
}
return multiGetRequestBuilder;
}
private static void assertResponseItems(List<BulkItemResponse> bulkItemResponses, int numDocs) {
assertThat(bulkItemResponses.size(), is(numDocs));
int i = 1;
for (BulkItemResponse bulkItemResponse : bulkItemResponses) {
assertThat(bulkItemResponse.getIndex(), equalTo("test"));
assertThat(bulkItemResponse.getType(), equalTo("test"));
assertThat(bulkItemResponse.getId(), equalTo(Integer.toString(i++)));
assertThat(bulkItemResponse.isFailed(), equalTo(false));
}
}
private static void assertMultiGetResponse(MultiGetResponse multiGetResponse, int numDocs) {
assertThat(multiGetResponse.getResponses().length, equalTo(numDocs));
int i = 1;
for (MultiGetItemResponse multiGetItemResponse : multiGetResponse) {
assertThat(multiGetItemResponse.getIndex(), equalTo("test"));
assertThat(multiGetItemResponse.getType(), equalTo("test"));
assertThat(multiGetItemResponse.getId(), equalTo(Integer.toString(i++)));
}
}
private static class BulkProcessorTestListener implements BulkProcessor.Listener {
private final CountDownLatch[] latches;
private final AtomicInteger beforeCounts = new AtomicInteger();
private final AtomicInteger afterCounts = new AtomicInteger();
private final List<BulkItemResponse> bulkItems = new CopyOnWriteArrayList<>();
private final List<Throwable> bulkFailures = new CopyOnWriteArrayList<>();
private BulkProcessorTestListener(CountDownLatch... latches) {
this.latches = latches;
}
@Override
public void beforeBulk(long executionId, BulkRequest request) {
beforeCounts.incrementAndGet();
}
@Override
public void afterBulk(long executionId, BulkRequest request, BulkResponse response) {
bulkItems.addAll(Arrays.asList(response.getItems()));
afterCounts.incrementAndGet();
for (CountDownLatch latch : latches) {
latch.countDown();
}
}
@Override
public void afterBulk(long executionId, BulkRequest request, Throwable failure) {
bulkFailures.add(failure);
afterCounts.incrementAndGet();
for (CountDownLatch latch : latches) {
latch.countDown();
}
}
}
}

View File

@ -20,14 +20,10 @@
package org.elasticsearch.document;
import com.google.common.base.Charsets;
import com.google.common.collect.Maps;
import org.elasticsearch.action.bulk.BulkProcessor;
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.count.CountResponse;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.update.UpdateRequestBuilder;
@ -41,10 +37,7 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
@ -576,50 +569,6 @@ public class BulkTests extends ElasticsearchIntegrationTest {
assertExists(get("test", "type", "48"));
}
@Test
public void testThatBulkProcessorCountIsCorrect() throws InterruptedException {
final AtomicReference<BulkResponse> responseRef = new AtomicReference<>();
final AtomicReference<Throwable> failureRef = new AtomicReference<>();
final CountDownLatch latch = new CountDownLatch(1);
BulkProcessor.Listener listener = new BulkProcessor.Listener() {
@Override
public void beforeBulk(long executionId, BulkRequest request) {
}
@Override
public void afterBulk(long executionId, BulkRequest request, BulkResponse response) {
responseRef.set(response);
latch.countDown();
}
@Override
public void afterBulk(long executionId, BulkRequest request, Throwable failure) {
failureRef.set(failure);
latch.countDown();
}
};
try (BulkProcessor processor = BulkProcessor.builder(client(), listener).setBulkActions(5)
.setConcurrentRequests(1).setName("foo").build()) {
Map<String, Object> data = Maps.newHashMap();
data.put("foo", "bar");
processor.add(new IndexRequest("test", "test", "1").source(data));
processor.add(new IndexRequest("test", "test", "2").source(data));
processor.add(new IndexRequest("test", "test", "3").source(data));
processor.add(new IndexRequest("test", "test", "4").source(data));
processor.add(new IndexRequest("test", "test", "5").source(data));
latch.await();
BulkResponse response = responseRef.get();
Throwable error = failureRef.get();
assertThat(error, nullValue());
assertThat("Could not get a bulk response even after an explicit flush.", response, notNullValue());
assertThat(response.getItems().length, is(5));
}
}
@Test // issue 4987
public void testThatInvalidIndexNamesShouldNotBreakCompleteBulkRequest() {
int bulkEntryCount = randomIntBetween(10, 50);
@ -647,49 +596,5 @@ public class BulkTests extends ElasticsearchIntegrationTest {
assertThat(bulkResponse.getItems()[i].isFailed(), is(expectedFailures[i]));
}
}
@Test
public void testBulkProcessorFlush() throws InterruptedException {
final AtomicReference<BulkResponse> responseRef = new AtomicReference<>();
final AtomicReference<Throwable> failureRef = new AtomicReference<>();
final CountDownLatch latch = new CountDownLatch(1);
BulkProcessor.Listener listener = new BulkProcessor.Listener() {
@Override
public void beforeBulk(long executionId, BulkRequest request) {
}
@Override
public void afterBulk(long executionId, BulkRequest request, BulkResponse response) {
responseRef.set(response);
latch.countDown();
}
@Override
public void afterBulk(long executionId, BulkRequest request, Throwable failure) {
failureRef.set(failure);
latch.countDown();
}
};
try (BulkProcessor processor = BulkProcessor.builder(client(), listener).setBulkActions(6)
.setConcurrentRequests(1).setName("foo").build()) {
Map<String, Object> data = Maps.newHashMap();
data.put("foo", "bar");
processor.add(new IndexRequest("test", "test", "1").source(data));
processor.add(new IndexRequest("test", "test", "2").source(data));
processor.add(new IndexRequest("test", "test", "3").source(data));
processor.add(new IndexRequest("test", "test", "4").source(data));
processor.add(new IndexRequest("test", "test", "5").source(data));
processor.flush();
latch.await();
BulkResponse response = responseRef.get();
Throwable error = failureRef.get();
assertThat(error, nullValue());
assertThat("Could not get a bulk response even after an explicit flush.", response, notNullValue());
assertThat(response.getItems().length, is(5));
}
}
}