Skip to content

Commit

Permalink
River name re-use not possible between node shutdowns, closes #1921.
Browse files Browse the repository at this point in the history
This effectively happens because we don't flush before deleting a mapping, which causes it to still reply the changes happening for it on the transaction log, meaning the mapping are still around on the index level.
  • Loading branch information
kimchy committed Jun 23, 2012
1 parent 6fb836c commit ded5b77
Showing 1 changed file with 24 additions and 8 deletions.
Expand Up @@ -21,6 +21,8 @@

import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.admin.indices.flush.TransportFlushAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.refresh.TransportRefreshAction;
import org.elasticsearch.action.deletebyquery.DeleteByQueryResponse;
Expand Down Expand Up @@ -49,18 +51,21 @@ public class TransportDeleteMappingAction extends TransportMasterNodeOperationAc

private final MetaDataMappingService metaDataMappingService;

private final TransportFlushAction flushAction;

private final TransportDeleteByQueryAction deleteByQueryAction;

private final TransportRefreshAction refreshAction;

@Inject
public TransportDeleteMappingAction(Settings settings, TransportService transportService, ClusterService clusterService,
ThreadPool threadPool, MetaDataMappingService metaDataMappingService,
TransportDeleteByQueryAction deleteByQueryAction, TransportRefreshAction refreshAction) {
TransportDeleteByQueryAction deleteByQueryAction, TransportRefreshAction refreshAction, TransportFlushAction flushAction) {
super(settings, transportService, clusterService, threadPool);
this.metaDataMappingService = metaDataMappingService;
this.deleteByQueryAction = deleteByQueryAction;
this.refreshAction = refreshAction;
this.flushAction = flushAction;
}

@Override
Expand Down Expand Up @@ -100,19 +105,30 @@ protected DeleteMappingResponse masterOperation(final DeleteMappingRequest reque

final AtomicReference<Throwable> failureRef = new AtomicReference<Throwable>();
final CountDownLatch latch = new CountDownLatch(1);
deleteByQueryAction.execute(Requests.deleteByQueryRequest(request.indices()).query(QueryBuilders.filteredQuery(QueryBuilders.matchAllQuery(), FilterBuilders.typeFilter(request.type()))), new ActionListener<DeleteByQueryResponse>() {
flushAction.execute(Requests.flushRequest(request.indices()), new ActionListener<FlushResponse>() {
@Override
public void onResponse(DeleteByQueryResponse deleteByQueryResponse) {
refreshAction.execute(Requests.refreshRequest(request.indices()), new ActionListener<RefreshResponse>() {
public void onResponse(FlushResponse flushResponse) {
deleteByQueryAction.execute(Requests.deleteByQueryRequest(request.indices()).query(QueryBuilders.filteredQuery(QueryBuilders.matchAllQuery(), FilterBuilders.typeFilter(request.type()))), new ActionListener<DeleteByQueryResponse>() {
@Override
public void onResponse(RefreshResponse refreshResponse) {
metaDataMappingService.removeMapping(new MetaDataMappingService.RemoveRequest(request.indices(), request.type()));
latch.countDown();
public void onResponse(DeleteByQueryResponse deleteByQueryResponse) {
refreshAction.execute(Requests.refreshRequest(request.indices()), new ActionListener<RefreshResponse>() {
@Override
public void onResponse(RefreshResponse refreshResponse) {
metaDataMappingService.removeMapping(new MetaDataMappingService.RemoveRequest(request.indices(), request.type()));
latch.countDown();
}

@Override
public void onFailure(Throwable e) {
metaDataMappingService.removeMapping(new MetaDataMappingService.RemoveRequest(request.indices(), request.type()));
latch.countDown();
}
});
}

@Override
public void onFailure(Throwable e) {
metaDataMappingService.removeMapping(new MetaDataMappingService.RemoveRequest(request.indices(), request.type()));
failureRef.set(e);
latch.countDown();
}
});
Expand Down

0 comments on commit ded5b77

Please sign in to comment.