move persistenting state and sharsd state in local gateway to another thread

This commit is contained in:
kimchy 2010-09-19 17:54:14 +02:00
parent f82ceb1e1e
commit 589dbce89b
1 changed files with 101 additions and 79 deletions

View File

@ -49,11 +49,15 @@ import org.elasticsearch.index.gateway.local.LocalIndexGatewayModule;
import java.io.*; import java.io.*;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import static java.util.concurrent.Executors.*;
import static org.elasticsearch.cluster.ClusterState.*; import static org.elasticsearch.cluster.ClusterState.*;
import static org.elasticsearch.cluster.metadata.MetaData.*; import static org.elasticsearch.cluster.metadata.MetaData.*;
import static org.elasticsearch.common.unit.TimeValue.*; import static org.elasticsearch.common.unit.TimeValue.*;
import static org.elasticsearch.common.util.concurrent.EsExecutors.*;
/** /**
* @author kimchy (shay.banon) * @author kimchy (shay.banon)
@ -78,6 +82,8 @@ public class LocalGateway extends AbstractLifecycleComponent<Gateway> implements
private volatile LocalGatewayStartedShards currentStartedShards; private volatile LocalGatewayStartedShards currentStartedShards;
private volatile ExecutorService executor;
private volatile boolean initialized = false; private volatile boolean initialized = false;
@Inject public LocalGateway(Settings settings, ClusterService clusterService, MetaDataCreateIndexService createIndexService, @Inject public LocalGateway(Settings settings, ClusterService clusterService, MetaDataCreateIndexService createIndexService,
@ -105,12 +111,19 @@ public class LocalGateway extends AbstractLifecycleComponent<Gateway> implements
} }
@Override protected void doStart() throws ElasticSearchException { @Override protected void doStart() throws ElasticSearchException {
this.executor = newSingleThreadExecutor(daemonThreadFactory(settings, "gateway"));
lazyInitialize(); lazyInitialize();
clusterService.add(this); clusterService.add(this);
} }
@Override protected void doStop() throws ElasticSearchException { @Override protected void doStop() throws ElasticSearchException {
clusterService.remove(this); clusterService.remove(this);
executor.shutdown();
try {
executor.awaitTermination(10, TimeUnit.SECONDS);
} catch (InterruptedException e) {
// ignore
}
} }
@Override protected void doClose() throws ElasticSearchException { @Override protected void doClose() throws ElasticSearchException {
@ -234,6 +247,8 @@ public class LocalGateway extends AbstractLifecycleComponent<Gateway> implements
} }
if (event.state().nodes().localNode().masterNode() && event.metaDataChanged()) { if (event.state().nodes().localNode().masterNode() && event.metaDataChanged()) {
executor.execute(new Runnable() {
@Override public void run() {
LocalGatewayMetaState.Builder builder = LocalGatewayMetaState.builder(); LocalGatewayMetaState.Builder builder = LocalGatewayMetaState.builder();
if (currentMetaState != null) { if (currentMetaState != null) {
builder.state(currentMetaState); builder.state(currentMetaState);
@ -272,8 +287,12 @@ public class LocalGateway extends AbstractLifecycleComponent<Gateway> implements
logger.warn("failed to write updated state", e); logger.warn("failed to write updated state", e);
} }
} }
});
}
if (event.state().nodes().localNode().dataNode() && event.routingTableChanged()) { if (event.state().nodes().localNode().dataNode() && event.routingTableChanged()) {
executor.execute(new Runnable() {
@Override public void run() {
LocalGatewayStartedShards.Builder builder = LocalGatewayStartedShards.builder(); LocalGatewayStartedShards.Builder builder = LocalGatewayStartedShards.builder();
if (currentStartedShards != null) { if (currentStartedShards != null) {
builder.state(currentStartedShards); builder.state(currentStartedShards);
@ -316,6 +335,7 @@ public class LocalGateway extends AbstractLifecycleComponent<Gateway> implements
currentStartedShards = stateToWrite; currentStartedShards = stateToWrite;
} catch (IOException e) { } catch (IOException e) {
logger.warn("failed to write updated state", e); logger.warn("failed to write updated state", e);
return;
} }
// delete all the other files // delete all the other files
@ -328,6 +348,8 @@ public class LocalGateway extends AbstractLifecycleComponent<Gateway> implements
file.delete(); file.delete();
} }
} }
});
}
} }
/** /**